You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by vg...@apache.org on 2018/05/15 22:40:21 UTC

[01/50] [abbrv] hive git commit: HIVE-19357: Vectorization: assert_true HiveException erroneously gets suppressed to NULL (Matt McCline, reviewed by Zoltan Haindrich)

Repository: hive
Updated Branches:
  refs/heads/branch-3 [deleted] f1f265046
  refs/heads/branch-3.0.0 [created] f1f265046


http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/udf/VectorUDFAdaptor.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/udf/VectorUDFAdaptor.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/udf/VectorUDFAdaptor.java
index 82b7a15..5b2cb4c 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/udf/VectorUDFAdaptor.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/udf/VectorUDFAdaptor.java
@@ -60,6 +60,7 @@ public class VectorUDFAdaptor extends VectorExpression {
   private String resultType;
   private VectorUDFArgDesc[] argDescs;
   private ExprNodeGenericFuncDesc expr;
+  private boolean suppressEvaluateExceptions;
 
   private transient GenericUDF genericUDF;
   private transient GenericUDF.DeferredObject[] deferredChildren;
@@ -84,6 +85,10 @@ public class VectorUDFAdaptor extends VectorExpression {
     this.argDescs = argDescs;
   }
 
+  public void setSuppressEvaluateExceptions(boolean suppressEvaluateExceptions) {
+    this.suppressEvaluateExceptions = suppressEvaluateExceptions;
+  }
+
   // Initialize transient fields. To be called after deserialization of other fields.
   public void init() throws HiveException, UDFArgumentException {
     genericUDF = expr.getGenericUDF();
@@ -112,7 +117,7 @@ public class VectorUDFAdaptor extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     if (genericUDF == null) {
       try {
@@ -192,7 +197,7 @@ public class VectorUDFAdaptor extends VectorExpression {
   /* Calculate the function result for row i of the batch and
    * set the output column vector entry i to the result.
    */
-  private void setResult(int i, VectorizedRowBatch b) {
+  private void setResult(int i, VectorizedRowBatch b) throws HiveException {
 
     // get arguments
     for (int j = 0; j < argDescs.length; j++) {
@@ -201,15 +206,19 @@ public class VectorUDFAdaptor extends VectorExpression {
 
     // call function
     Object result;
-    try {
+    if (!suppressEvaluateExceptions) {
       result = genericUDF.evaluate(deferredChildren);
-    } catch (HiveException e) {
-
-      /* For UDFs that expect primitive types (like int instead of Integer or IntWritable),
-       * this will catch the the exception that happens if they are passed a NULL value.
-       * Then the default NULL handling logic will apply, and the result will be NULL.
-       */
-      result = null;
+    } else {
+      try {
+        result = genericUDF.evaluate(deferredChildren);
+      } catch (HiveException e) {
+
+        /* For UDFs that expect primitive types (like int instead of Integer or IntWritable),
+         * this will catch the the exception that happens if they are passed a NULL value.
+         * Then the default NULL handling logic will apply, and the result will be NULL.
+         */
+        result = null;
+      }
     }
 
     // Set output column vector entry.  Since we have one output column, the logical index = 0.

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestUnaryMinus.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestUnaryMinus.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestUnaryMinus.java
index e227f44..3860345 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestUnaryMinus.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestUnaryMinus.java
@@ -25,6 +25,7 @@ import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.LongColUnaryMinus;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.LongColUnaryMinusChecked;
 import org.apache.hadoop.hive.ql.exec.vector.util.VectorizedRowGroupGenUtil;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
 import org.junit.Test;
 
@@ -34,7 +35,7 @@ import org.junit.Test;
 public class TestUnaryMinus {
 
   @Test
-  public void testUnaryMinus() {
+  public void testUnaryMinus() throws HiveException {
     VectorizedRowBatch vrg = VectorizedRowGroupGenUtil.getVectorizedRowBatch(1024, 2, 23);
     LongColUnaryMinus expr = new LongColUnaryMinus(0, 1);
     expr.evaluate(vrg);
@@ -48,7 +49,7 @@ public class TestUnaryMinus {
 
 
   @Test
-  public void testUnaryMinusCheckedOverflow() {
+  public void testUnaryMinusCheckedOverflow() throws HiveException {
     VectorizedRowBatch vrg = VectorizedRowGroupGenUtil.getVectorizedRowBatch(1, 2, 0);
     //set value to MIN_VALUE so that -MIN_VALUE overflows and gets set to MIN_VALUE again
     ((LongColumnVector)vrg.cols[0]).vector[0] = Integer.MIN_VALUE;
@@ -64,7 +65,7 @@ public class TestUnaryMinus {
   }
 
   @Test
-  public void testUnaryMinusChecked() {
+  public void testUnaryMinusChecked() throws HiveException {
     VectorizedRowBatch vrg = VectorizedRowGroupGenUtil.getVectorizedRowBatch(1024, 2, 23);
     LongColUnaryMinusChecked expr = new LongColUnaryMinusChecked(0, 1);
     expr.setOutputTypeInfo(TypeInfoFactory.getPrimitiveTypeInfo("bigint"));

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorArithmeticExpressions.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorArithmeticExpressions.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorArithmeticExpressions.java
index f2adc08..f5491af 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorArithmeticExpressions.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorArithmeticExpressions.java
@@ -55,6 +55,7 @@ import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.DecimalScalarSubtra
 import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.DecimalScalarMultiplyDecimalColumn;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.LongColAddLongScalarChecked;
 import org.apache.hadoop.hive.ql.exec.vector.util.VectorizedRowGroupGenUtil;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
 import org.junit.Test;
 
@@ -64,16 +65,16 @@ import org.junit.Test;
 public class TestVectorArithmeticExpressions {
 
   @Test
-  public void testLongColAddLongScalarNoNulls()  {
+  public void testLongColAddLongScalarNoNulls() throws HiveException {
     longColAddLongScalarNoNulls(false);
   }
 
   @Test
-  public void testLongColAddLongScalarCheckedNoNulls()  {
+  public void testLongColAddLongScalarCheckedNoNulls() throws HiveException {
     longColAddLongScalarNoNulls(true);
   }
 
-  private void longColAddLongScalarNoNulls(boolean checked)  {
+  private void longColAddLongScalarNoNulls(boolean checked) throws HiveException {
     VectorizedRowBatch vrg = getVectorizedRowBatchSingleLongVector(VectorizedRowBatch.DEFAULT_SIZE);
     VectorExpression expr;
     if (checked) {
@@ -122,16 +123,16 @@ public class TestVectorArithmeticExpressions {
   }
 
   @Test
-  public void testLongColAddLongScalarWithNulls()  {
+  public void testLongColAddLongScalarWithNulls() throws HiveException {
     longColAddLongScalarCheckedWithNulls(false);
   }
 
   @Test
-  public void testLongColAddLongScalarCheckedWithNulls()  {
+  public void testLongColAddLongScalarCheckedWithNulls() throws HiveException {
     longColAddLongScalarCheckedWithNulls(true);
   }
 
-  private void longColAddLongScalarCheckedWithNulls(boolean isChecked)  {
+  private void longColAddLongScalarCheckedWithNulls(boolean isChecked) throws HiveException {
     VectorizedRowBatch batch = getVectorizedRowBatchSingleLongVector(
         VectorizedRowBatch.DEFAULT_SIZE);
     LongColumnVector lcv = (LongColumnVector) batch.cols[0];
@@ -160,16 +161,16 @@ public class TestVectorArithmeticExpressions {
   }
 
   @Test
-  public void testLongColAddLongScalarWithRepeating() {
+  public void testLongColAddLongScalarWithRepeating() throws HiveException {
     longColAddLongScalarWithRepeatingUtil(false);
   }
 
   @Test
-  public void testLongColAddLongScalarCheckedWithRepeating() {
+  public void testLongColAddLongScalarCheckedWithRepeating() throws HiveException {
     longColAddLongScalarWithRepeatingUtil(true);
   }
 
-  private void longColAddLongScalarWithRepeatingUtil(boolean isChecked) {
+  private void longColAddLongScalarWithRepeatingUtil(boolean isChecked) throws HiveException {
     LongColumnVector in, out;
     VectorizedRowBatch batch;
     VectorExpression expr;
@@ -248,16 +249,16 @@ public class TestVectorArithmeticExpressions {
   }
 
   @Test
-  public void testLongColAddLongColumn() {
+  public void testLongColAddLongColumn() throws HiveException {
     longColAddLongColumnUtil(false);
   }
 
   @Test
-  public void testLongColAddLongColumnChecked() {
+  public void testLongColAddLongColumnChecked() throws HiveException {
     longColAddLongColumnUtil(true);
   }
 
-  private void longColAddLongColumnUtil(boolean isChecked) {
+  private void longColAddLongColumnUtil(boolean isChecked) throws HiveException {
     int seed = 17;
     VectorizedRowBatch vrg = VectorizedRowGroupGenUtil.getVectorizedRowBatch(
         VectorizedRowBatch.DEFAULT_SIZE,
@@ -348,7 +349,7 @@ public class TestVectorArithmeticExpressions {
   }
 
   @Test
-  public void testLongColDivideLongColumn() {
+  public void testLongColDivideLongColumn() throws HiveException {
     /* Testing for equality of doubles after a math operation is
      * not always reliable so use this as a tolerance.
      */
@@ -384,7 +385,7 @@ public class TestVectorArithmeticExpressions {
   }
 
   @Test
-  public void testLongColModuloLongColumn() {
+  public void testLongColModuloLongColumn() throws HiveException {
     VectorizedRowBatch batch = getVectorizedRowBatch2LongInLongOut();
     LongColModuloLongColumn expr = new LongColModuloLongColumn(0, 1, 2);
     batch.cols[0].isNull[1] = true;
@@ -432,7 +433,7 @@ public class TestVectorArithmeticExpressions {
   }
 
   @Test
-  public void testDecimalColAddDecimalColumn() {
+  public void testDecimalColAddDecimalColumn() throws HiveException {
     VectorizedRowBatch b = getVectorizedRowBatch3DecimalCols();
     VectorExpression expr = new DecimalColAddDecimalColumn(0, 1, 2);
     DecimalColumnVector r = (DecimalColumnVector) b.cols[2];
@@ -496,7 +497,7 @@ public class TestVectorArithmeticExpressions {
 
   // Spot check decimal column-column subtract
   @Test
-  public void testDecimalColSubtractDecimalColumn() {
+  public void testDecimalColSubtractDecimalColumn() throws HiveException {
     VectorizedRowBatch b = getVectorizedRowBatch3DecimalCols();
     VectorExpression expr = new DecimalColSubtractDecimalColumn(0, 1, 2);
     DecimalColumnVector r = (DecimalColumnVector) b.cols[2];
@@ -518,7 +519,7 @@ public class TestVectorArithmeticExpressions {
 
   // Spot check decimal column-column multiply
   @Test
-  public void testDecimalColMultiplyDecimalColumn() {
+  public void testDecimalColMultiplyDecimalColumn() throws HiveException {
     VectorizedRowBatch b = getVectorizedRowBatch3DecimalCols();
     VectorExpression expr = new DecimalColMultiplyDecimalColumn(0, 1, 2);
     DecimalColumnVector r = (DecimalColumnVector) b.cols[2];
@@ -544,7 +545,7 @@ public class TestVectorArithmeticExpressions {
    * cases used in the source code template ColumnArithmeticScalarDecimal.txt.
    */
   @Test
-  public void testDecimalColAddDecimalScalar() {
+  public void testDecimalColAddDecimalScalar() throws HiveException {
     VectorizedRowBatch b = getVectorizedRowBatch3DecimalCols();
     HiveDecimal d = HiveDecimal.create(1);
     VectorExpression expr = new DecimalColAddDecimalScalar(0, d, 2);
@@ -602,7 +603,7 @@ public class TestVectorArithmeticExpressions {
    * The template is used for division and modulo.
    */
   @Test
-  public void testDecimalColDivideDecimalScalar() {
+  public void testDecimalColDivideDecimalScalar() throws HiveException {
     VectorizedRowBatch b = getVectorizedRowBatch3DecimalCols();
     HiveDecimal d = HiveDecimal.create("2.00");
     VectorExpression expr = new DecimalColDivideDecimalScalar(0, d, 2);
@@ -661,7 +662,7 @@ public class TestVectorArithmeticExpressions {
    * for template ScalarDivideColumnDecimal.txt.
    */
   @Test
-  public void testDecimalScalarDivideDecimalColumn() {
+  public void testDecimalScalarDivideDecimalColumn() throws HiveException {
     VectorizedRowBatch b = getVectorizedRowBatch3DecimalCols();
     HiveDecimal d = HiveDecimal.create("3.96");  // 1.20 * 3.30
     VectorExpression expr = new DecimalScalarDivideDecimalColumn(d, 0, 2);
@@ -708,7 +709,7 @@ public class TestVectorArithmeticExpressions {
 
   // Spot check Decimal Col-Scalar Modulo
   @Test
-  public void testDecimalColModuloDecimalScalar() {
+  public void testDecimalColModuloDecimalScalar() throws HiveException {
     VectorizedRowBatch b = getVectorizedRowBatch3DecimalCols();
     HiveDecimal d = HiveDecimal.create("2.00");
     VectorExpression expr = new DecimalColModuloDecimalScalar(0, d, 2);
@@ -744,7 +745,7 @@ public class TestVectorArithmeticExpressions {
 
   // Spot check decimal scalar-column modulo
   @Test
-  public void testDecimalScalarModuloDecimalColumn() {
+  public void testDecimalScalarModuloDecimalColumn() throws HiveException {
     VectorizedRowBatch b = getVectorizedRowBatch3DecimalCols();
     HiveDecimal d = HiveDecimal.create("2.00");
     VectorExpression expr = new DecimalScalarModuloDecimalColumn(d, 0, 2);
@@ -771,7 +772,7 @@ public class TestVectorArithmeticExpressions {
   }
 
   @Test
-  public void testDecimalColDivideDecimalColumn() {
+  public void testDecimalColDivideDecimalColumn() throws HiveException {
     VectorizedRowBatch b = getVectorizedRowBatch3DecimalCols();
     DecimalColumnVector in1 = (DecimalColumnVector) b.cols[1];
     for (int i = 0; i < 3; i++) {
@@ -848,7 +849,7 @@ public class TestVectorArithmeticExpressions {
 
   // Spot check decimal column modulo decimal column
   @Test
-  public void testDecimalColModuloDecimalColumn() {
+  public void testDecimalColModuloDecimalColumn() throws HiveException {
     VectorizedRowBatch b = getVectorizedRowBatch3DecimalCols();
     DecimalColumnVector in1 = (DecimalColumnVector) b.cols[1];
     for (int i = 0; i < 3; i++) {
@@ -867,7 +868,7 @@ public class TestVectorArithmeticExpressions {
    * addition checks all the cases for the template, so don't do that redundantly here.
    */
   @Test
-  public void testDecimalColSubtractDecimalScalar() {
+  public void testDecimalColSubtractDecimalScalar() throws HiveException {
     VectorizedRowBatch b = getVectorizedRowBatch3DecimalCols();
     HiveDecimal d = HiveDecimal.create(1);
     VectorExpression expr = new DecimalColSubtractDecimalScalar(0, d, 2);
@@ -893,7 +894,7 @@ public class TestVectorArithmeticExpressions {
    * addition checks all the cases for the template, so don't do that redundantly here.
    */
   @Test
-  public void testDecimalColMultiplyDecimalScalar() {
+  public void testDecimalColMultiplyDecimalScalar() throws HiveException {
     VectorizedRowBatch b = getVectorizedRowBatch3DecimalCols();
     HiveDecimal d = HiveDecimal.create(2);
     VectorExpression expr = new DecimalColMultiplyDecimalScalar(0, d, 2);
@@ -919,7 +920,7 @@ public class TestVectorArithmeticExpressions {
    * cases used in the source code template ScalarArithmeticColumnDecimal.txt.
    */
   @Test
-  public void testDecimalScalarAddDecimalColumn() {
+  public void testDecimalScalarAddDecimalColumn() throws HiveException {
     VectorizedRowBatch b = getVectorizedRowBatch3DecimalCols();
     HiveDecimal d = HiveDecimal.create(1);
     VectorExpression expr = new DecimalScalarAddDecimalColumn(d, 0, 2);
@@ -976,7 +977,7 @@ public class TestVectorArithmeticExpressions {
    * addition checks all the cases for the template, so don't do that redundantly here.
    */
   @Test
-  public void testDecimalScalarSubtractDecimalColumn() {
+  public void testDecimalScalarSubtractDecimalColumn() throws HiveException {
     VectorizedRowBatch b = getVectorizedRowBatch3DecimalCols();
     HiveDecimal d = HiveDecimal.create(1);
     VectorExpression expr = new DecimalScalarSubtractDecimalColumn(d, 0, 2);
@@ -1003,7 +1004,7 @@ public class TestVectorArithmeticExpressions {
    */
 
   @Test
-  public void testDecimalScalarMultiplyDecimalColumn() {
+  public void testDecimalScalarMultiplyDecimalColumn() throws HiveException {
     VectorizedRowBatch b = getVectorizedRowBatch3DecimalCols();
     HiveDecimal d = HiveDecimal.create(2);
     VectorExpression expr = new DecimalScalarMultiplyDecimalColumn(d, 0, 2);

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorConditionalExpressions.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorConditionalExpressions.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorConditionalExpressions.java
index ea19e93..d02ae02 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorConditionalExpressions.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorConditionalExpressions.java
@@ -35,7 +35,7 @@ import org.apache.hadoop.hive.ql.exec.vector.expressions.IfExprStringGroupColumn
 import org.apache.hadoop.hive.ql.exec.vector.expressions.IfExprStringGroupColumnStringScalar;
 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.metadata.HiveException;
 import org.junit.Test;
 
 /**
@@ -175,7 +175,7 @@ public class TestVectorConditionalExpressions {
   }
 
   @Test
-  public void testLongColumnColumnIfExpr()  {
+  public void testLongColumnColumnIfExpr() throws HiveException {
     VectorizedRowBatch batch = getBatch4LongVectors();
     VectorExpression expr = new IfExprLongColumnLongColumn(0, 1, 2, 3);
     expr.evaluate(batch);
@@ -290,7 +290,7 @@ public class TestVectorConditionalExpressions {
   }
 
   @Test
-  public void testDoubleColumnColumnIfExpr()  {
+  public void testDoubleColumnColumnIfExpr() throws HiveException {
     // Just spot check because we already checked the logic for long.
     // The code is from the same template file.
 
@@ -310,7 +310,7 @@ public class TestVectorConditionalExpressions {
   }
 
   @Test
-  public void testLongColumnScalarIfExpr() {
+  public void testLongColumnScalarIfExpr() throws HiveException {
     VectorizedRowBatch batch = getBatch4LongVectors();
     VectorExpression expr = new IfExprLongColumnLongScalar(0, 1, 100, 3);
     LongColumnVector r = (LongColumnVector) batch.cols[3];
@@ -322,7 +322,7 @@ public class TestVectorConditionalExpressions {
   }
 
   @Test
-  public void testLongScalarColumnIfExpr() {
+  public void testLongScalarColumnIfExpr() throws HiveException {
     VectorizedRowBatch batch = getBatch4LongVectors();
     VectorExpression expr = new IfExprLongScalarLongColumn(0, 100, 2, 3);
     LongColumnVector r = (LongColumnVector) batch.cols[3];
@@ -334,7 +334,7 @@ public class TestVectorConditionalExpressions {
   }
 
   @Test
-  public void testLongScalarScalarIfExpr() {
+  public void testLongScalarScalarIfExpr() throws HiveException {
     VectorizedRowBatch batch = getBatch4LongVectors();
     VectorExpression expr = new IfExprLongScalarLongScalar(0, 100, 200, 3);
     LongColumnVector r = (LongColumnVector) batch.cols[3];
@@ -346,7 +346,7 @@ public class TestVectorConditionalExpressions {
   }
 
   @Test
-  public void testDoubleScalarScalarIfExpr() {
+  public void testDoubleScalarScalarIfExpr() throws HiveException {
     VectorizedRowBatch batch = getBatch1Long3DoubleVectors();
     VectorExpression expr = new IfExprDoubleScalarDoubleScalar(0, 100.0d, 200.0d, 3);
     DoubleColumnVector r = (DoubleColumnVector) batch.cols[3];
@@ -358,7 +358,7 @@ public class TestVectorConditionalExpressions {
   }
 
   @Test
-  public void testDoubleScalarColumnIfExpr() {
+  public void testDoubleScalarColumnIfExpr() throws HiveException {
     VectorizedRowBatch batch = getBatch1Long3DoubleVectors();
     VectorExpression expr = new IfExprDoubleScalarDoubleColumn(0, 100.0d, 2, 3);
     DoubleColumnVector r = (DoubleColumnVector) batch.cols[3];
@@ -370,7 +370,7 @@ public class TestVectorConditionalExpressions {
   }
 
   @Test
-  public void testDoubleColumnScalarIfExpr() {
+  public void testDoubleColumnScalarIfExpr() throws HiveException {
     VectorizedRowBatch batch = getBatch1Long3DoubleVectors();
     VectorExpression expr = new IfExprDoubleColumnDoubleScalar(0, 1, 200d, 3);
     DoubleColumnVector r = (DoubleColumnVector) batch.cols[3];
@@ -382,7 +382,7 @@ public class TestVectorConditionalExpressions {
   }
 
   @Test
-  public void testIfExprStringColumnStringColumn() {
+  public void testIfExprStringColumnStringColumn() throws HiveException {
     VectorizedRowBatch batch = getBatch1Long3BytesVectors();
     VectorExpression expr = new IfExprStringGroupColumnStringGroupColumn(0, 1, 2, 3);
     BytesColumnVector r = (BytesColumnVector) batch.cols[3];
@@ -467,7 +467,7 @@ public class TestVectorConditionalExpressions {
   }
 
   @Test
-  public void testIfExprStringColumnStringScalar() {
+  public void testIfExprStringColumnStringScalar() throws HiveException {
     VectorizedRowBatch batch = getBatch1Long3BytesVectors();
     byte[] scalar = getUTF8Bytes("scalar");
     VectorExpression expr = new IfExprStringGroupColumnStringScalar(0, 1, scalar, 3);
@@ -490,7 +490,7 @@ public class TestVectorConditionalExpressions {
   }
 
   @Test
-  public void testIfExprStringScalarStringColumn() {
+  public void testIfExprStringScalarStringColumn() throws HiveException {
     VectorizedRowBatch batch = getBatch1Long3BytesVectors();
     byte[] scalar = getUTF8Bytes("scalar");
     VectorExpression expr = new IfExprStringScalarStringGroupColumn(0,scalar, 2, 3);
@@ -513,7 +513,7 @@ public class TestVectorConditionalExpressions {
   }
 
   @Test
-  public void testIfExprStringScalarStringScalar() {
+  public void testIfExprStringScalarStringScalar() throws HiveException {
 
     // standard case
     VectorizedRowBatch batch = getBatch1Long3BytesVectors();

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorDateExpressions.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorDateExpressions.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorDateExpressions.java
index 9db91be..ffe9c81 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorDateExpressions.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorDateExpressions.java
@@ -115,7 +115,7 @@ public class TestVectorDateExpressions {
     Assert.assertEquals(res.get(), y);
   }
 
-  private void verifyUDFYear(VectorizedRowBatch batch) {
+  private void verifyUDFYear(VectorizedRowBatch batch) throws HiveException {
     VectorExpression udf = null;
     udf = new VectorUDFYearDate(0, 1);
     udf.setInputTypeInfos(new TypeInfo[] {TypeInfoFactory.dateTypeInfo});
@@ -138,7 +138,7 @@ public class TestVectorDateExpressions {
   }
 
   @Test
-  public void testVectorUDFYear() {
+  public void testVectorUDFYear() throws HiveException {
     VectorizedRowBatch batch = getVectorizedRowBatch(new int[] {0},
             VectorizedRowBatch.DEFAULT_SIZE);
     Assert.assertTrue(((LongColumnVector) batch.cols[1]).noNulls);
@@ -177,7 +177,7 @@ public class TestVectorDateExpressions {
     Assert.assertEquals(res.get(), y);
   }
 
-  private void verifyUDFDayOfMonth(VectorizedRowBatch batch) {
+  private void verifyUDFDayOfMonth(VectorizedRowBatch batch) throws HiveException {
     VectorExpression udf = null;
     udf = new VectorUDFDayOfMonthDate(0, 1);
     udf.setInputTypeInfos(new TypeInfo[] {TypeInfoFactory.dateTypeInfo});
@@ -200,7 +200,7 @@ public class TestVectorDateExpressions {
   }
 
   @Test
-  public void testVectorUDFDayOfMonth() {
+  public void testVectorUDFDayOfMonth() throws HiveException {
     VectorizedRowBatch batch = getVectorizedRowBatch(new int[] {0},
             VectorizedRowBatch.DEFAULT_SIZE);
     Assert.assertTrue(((LongColumnVector) batch.cols[1]).noNulls);
@@ -239,7 +239,7 @@ public class TestVectorDateExpressions {
     Assert.assertEquals(res.get(), y);
   }
 
-  private void verifyUDFMonth(VectorizedRowBatch batch) {
+  private void verifyUDFMonth(VectorizedRowBatch batch) throws HiveException {
     VectorExpression udf;
     udf = new VectorUDFMonthDate(0, 1);
     udf.setInputTypeInfos(new TypeInfo[] {TypeInfoFactory.dateTypeInfo});
@@ -262,7 +262,7 @@ public class TestVectorDateExpressions {
   }
 
   @Test
-  public void testVectorUDFMonth() {
+  public void testVectorUDFMonth() throws HiveException {
     VectorizedRowBatch batch = getVectorizedRowBatch(new int[] {0},
             VectorizedRowBatch.DEFAULT_SIZE);
     Assert.assertTrue(((LongColumnVector) batch.cols[1]).noNulls);
@@ -315,7 +315,7 @@ public class TestVectorDateExpressions {
     Assert.assertEquals(res.get(), y);
   }
 
-  private void verifyUDFUnixTimeStamp(VectorizedRowBatch batch) {
+  private void verifyUDFUnixTimeStamp(VectorizedRowBatch batch) throws HiveException {
     VectorExpression udf;
     udf = new VectorUDFUnixTimeStampDate(0, 1);
     udf.setInputTypeInfos(new TypeInfo[] {TypeInfoFactory.dateTypeInfo});
@@ -338,7 +338,7 @@ public class TestVectorDateExpressions {
   }
 
   @Test
-  public void testVectorUDFUnixTimeStamp() {
+  public void testVectorUDFUnixTimeStamp() throws HiveException {
     VectorizedRowBatch batch = getVectorizedRowBatch(new int[] {0},
             VectorizedRowBatch.DEFAULT_SIZE);
     Assert.assertTrue(((LongColumnVector) batch.cols[1]).noNulls);
@@ -460,7 +460,7 @@ public class TestVectorDateExpressions {
 
   // 5s timeout
   @Test(timeout = 5000)
-  public void testMultiThreadedVectorUDFDate() {
+  public void testMultiThreadedVectorUDFDate() throws HiveException {
     List<Callable<Void>> tasks = new ArrayList<Callable<Void>>();
     for (int i = 0; i < 200; i++) {
       tasks.add(new MultiThreadedDateFormatTest());

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorFilterExpressions.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorFilterExpressions.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorFilterExpressions.java
index 3e769bb..5f4d138 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorFilterExpressions.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorFilterExpressions.java
@@ -65,7 +65,7 @@ import org.junit.Test;
 public class TestVectorFilterExpressions {
 
   @Test
-  public void testFilterLongColEqualLongScalar() {
+  public void testFilterLongColEqualLongScalar() throws HiveException {
     VectorizedRowBatch vrg =
         VectorizedRowGroupGenUtil.getVectorizedRowBatch(1024, 1, 23);
     FilterLongColEqualLongScalar expr = new FilterLongColEqualLongScalar(0, 46);
@@ -75,7 +75,7 @@ public class TestVectorFilterExpressions {
   }
 
   @Test
-  public void testFilterLongColGreaterLongColumn() {
+  public void testFilterLongColGreaterLongColumn() throws HiveException {
     int seed = 17;
     VectorizedRowBatch b = VectorizedRowGroupGenUtil.getVectorizedRowBatch(
         VectorizedRowBatch.DEFAULT_SIZE,
@@ -132,7 +132,7 @@ public class TestVectorFilterExpressions {
   }
 
   @Test
-  public void testColOpScalarNumericFilterNullAndRepeatingLogic() {
+  public void testColOpScalarNumericFilterNullAndRepeatingLogic() throws HiveException {
     // No nulls, not repeating
     FilterLongColGreaterLongScalar f = new FilterLongColGreaterLongScalar(0, 1);
     VectorizedRowBatch batch = this.getSimpleLongBatch();
@@ -196,7 +196,7 @@ public class TestVectorFilterExpressions {
   }
 
   @Test
-  public void testFilterLongColLessLongColumn() {
+  public void testFilterLongColLessLongColumn() throws HiveException {
     int seed = 17;
     VectorizedRowBatch vrg = VectorizedRowGroupGenUtil.getVectorizedRowBatch(
         5, 3, seed);
@@ -229,7 +229,7 @@ public class TestVectorFilterExpressions {
   }
 
   @Test
-  public void testFilterLongScalarLessLongColumn() {
+  public void testFilterLongScalarLessLongColumn() throws HiveException {
     int seed = 17;
     VectorizedRowBatch vrb = VectorizedRowGroupGenUtil.getVectorizedRowBatch(
         5, 2, seed);
@@ -333,7 +333,7 @@ public class TestVectorFilterExpressions {
   }
 
   @Test
-  public void testFilterLongBetween() {
+  public void testFilterLongBetween() throws HiveException {
     int seed = 17;
     VectorizedRowBatch vrb = VectorizedRowGroupGenUtil.getVectorizedRowBatch(
         5, 2, seed);
@@ -429,7 +429,7 @@ public class TestVectorFilterExpressions {
   }
 
   @Test
-  public void testFilterLongNotBetween() {
+  public void testFilterLongNotBetween() throws HiveException {
 
     // Spot check only. null & repeating behavior are checked elsewhere for the same template.
     int seed = 17;
@@ -452,7 +452,7 @@ public class TestVectorFilterExpressions {
   }
 
   @Test
-  public void testFilterDoubleBetween() {
+  public void testFilterDoubleBetween() throws HiveException {
 
     // Spot check only. null & repeating behavior are checked elsewhere for the same template.
     int seed = 17;
@@ -476,7 +476,7 @@ public class TestVectorFilterExpressions {
   }
 
   @Test
-  public void testFilterDoubleNotBetween() {
+  public void testFilterDoubleNotBetween() throws HiveException {
 
     // Spot check only. null & repeating behavior are checked elsewhere for the same template.
     int seed = 17;
@@ -514,7 +514,7 @@ public class TestVectorFilterExpressions {
   }
 
   @Test
-  public void testFilterStringBetween() {
+  public void testFilterStringBetween() throws HiveException {
     int seed = 17;
     VectorizedRowBatch vrb = VectorizedRowGroupGenUtil.getVectorizedRowBatch(
         3, 2, seed);
@@ -566,7 +566,7 @@ public class TestVectorFilterExpressions {
   }
 
   @Test
-  public void testFilterStringNotBetween() {
+  public void testFilterStringNotBetween() throws HiveException {
 
     // Spot check only. Non-standard cases are checked for the same template in another test.
     int seed = 17;
@@ -589,7 +589,7 @@ public class TestVectorFilterExpressions {
   }
 
   @Test
-  public void testFilterTimestampBetween() {
+  public void testFilterTimestampBetween() throws HiveException {
 
     VectorizedRowBatch vrb = new VectorizedRowBatch(1);
     vrb.cols[0] = new TimestampColumnVector();
@@ -614,7 +614,7 @@ public class TestVectorFilterExpressions {
   }
 
   @Test
-  public void testFilterTimestampNotBetween() {
+  public void testFilterTimestampNotBetween() throws HiveException {
     VectorizedRowBatch vrb = new VectorizedRowBatch(1);
     vrb.cols[0] = new TimestampColumnVector();
 
@@ -775,7 +775,7 @@ public class TestVectorFilterExpressions {
   }
 
   @Test
-  public void testFilterStringIn() {
+  public void testFilterStringIn() throws HiveException {
     int seed = 17;
     VectorizedRowBatch vrb = VectorizedRowGroupGenUtil.getVectorizedRowBatch(
         3, 2, seed);
@@ -834,7 +834,7 @@ public class TestVectorFilterExpressions {
    * multiple cases because the logic is the same for <, >, <=, >=, == and !=.
    */
   @Test
-  public void testFilterDecimalColEqualDecimalScalar() {
+  public void testFilterDecimalColEqualDecimalScalar() throws HiveException {
     VectorizedRowBatch b = getVectorizedRowBatch1DecimalCol();
     HiveDecimal scalar = HiveDecimal.create("-3.30");
     VectorExpression expr = new FilterDecimalColEqualDecimalScalar(0, scalar);
@@ -879,7 +879,7 @@ public class TestVectorFilterExpressions {
    * cases because the logic is the same for <, >, <=, >=, == and !=.
    */
   @Test
-  public void testFilterDecimalScalarEqualDecimalColumn() {
+  public void testFilterDecimalScalarEqualDecimalColumn() throws HiveException {
     VectorizedRowBatch b = getVectorizedRowBatch1DecimalCol();
     HiveDecimal scalar = HiveDecimal.create("-3.30");
     VectorExpression expr = new FilterDecimalScalarEqualDecimalColumn(scalar, 0);
@@ -924,7 +924,7 @@ public class TestVectorFilterExpressions {
    * cases because the logic is the same for <, >, <=, >=, == and !=.
    */
   @Test
-  public void testFilterDecimalColumnEqualDecimalColumn() {
+  public void testFilterDecimalColumnEqualDecimalColumn() throws HiveException {
     VectorizedRowBatch b = getVectorizedRowBatch2DecimalCol();
     VectorExpression expr = new FilterDecimalColEqualDecimalColumn(0, 1);
     expr.evaluate(b);
@@ -984,7 +984,7 @@ public class TestVectorFilterExpressions {
    * Spot check col < scalar for decimal.
    */
   @Test
-  public void testFilterDecimalColLessScalar() {
+  public void testFilterDecimalColLessScalar() throws HiveException {
     VectorizedRowBatch b = getVectorizedRowBatch1DecimalCol();
     HiveDecimal scalar = HiveDecimal.create("0");
     VectorExpression expr = new FilterDecimalColLessDecimalScalar(0, scalar);
@@ -1000,7 +1000,7 @@ public class TestVectorFilterExpressions {
    * Spot check scalar > col for decimal.
    */
   @Test
-  public void testFilterDecimalScalarGreaterThanColumn() {
+  public void testFilterDecimalScalarGreaterThanColumn() throws HiveException {
     VectorizedRowBatch b = getVectorizedRowBatch1DecimalCol();
     HiveDecimal scalar = HiveDecimal.create("0");
     VectorExpression expr = new FilterDecimalScalarGreaterDecimalColumn(scalar, 0);
@@ -1016,7 +1016,7 @@ public class TestVectorFilterExpressions {
    * Spot check col >= col for decimal.
    */
   @Test
-  public void testFilterDecimalColGreaterEqualCol() {
+  public void testFilterDecimalColGreaterEqualCol() throws HiveException {
     VectorizedRowBatch b = getVectorizedRowBatch2DecimalCol();
     VectorExpression expr = new FilterDecimalColGreaterEqualDecimalColumn(0, 1);
     expr.evaluate(b);

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorLogicalExpressions.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorLogicalExpressions.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorLogicalExpressions.java
index a60b9e4..50ad196 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorLogicalExpressions.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorLogicalExpressions.java
@@ -26,6 +26,7 @@ import org.apache.hadoop.hive.ql.exec.vector.DoubleColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor.Descriptor;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.junit.Assert;
 import org.junit.Test;
 
@@ -37,7 +38,7 @@ public class TestVectorLogicalExpressions {
   private static final int BOOLEAN_COLUMN_TEST_SIZE = 9;
 
   @Test
-  public void testLongColOrLongCol() {
+  public void testLongColOrLongCol() throws HiveException {
     VectorizedRowBatch batch = getBatchThreeBooleanCols();
     ColOrCol expr = new ColOrCol(0, 1, 2);
     LongColumnVector outCol = (LongColumnVector) batch.cols[2];
@@ -92,7 +93,7 @@ public class TestVectorLogicalExpressions {
   }
 
   @Test
-  public void testLongColAndLongCol() {
+  public void testLongColAndLongCol() throws HiveException {
     VectorizedRowBatch batch = getBatchThreeBooleanCols();
     ColAndCol expr = new ColAndCol(0, 1, 2);
     LongColumnVector outCol = (LongColumnVector) batch.cols[2];
@@ -187,7 +188,7 @@ public class TestVectorLogicalExpressions {
   }
 
   @Test
-  public void testBooleanNot() {
+  public void testBooleanNot() throws HiveException {
     VectorizedRowBatch batch = getBatchThreeBooleanCols();
     NotCol expr = new NotCol(0, 2);
     LongColumnVector outCol = (LongColumnVector) batch.cols[2];
@@ -229,7 +230,7 @@ public class TestVectorLogicalExpressions {
   }
 
   @Test
-  public void testIsNullExpr() {
+  public void testIsNullExpr() throws HiveException {
     // has nulls, not repeating
     VectorizedRowBatch batch = getBatchThreeBooleanCols();
     IsNull expr = new IsNull(0, 2);
@@ -269,7 +270,7 @@ public class TestVectorLogicalExpressions {
   }
 
   @Test
-  public void testIsNotNullExpr() {
+  public void testIsNotNullExpr() throws HiveException {
     // has nulls, not repeating
     VectorizedRowBatch batch = getBatchThreeBooleanCols();
     IsNotNull expr = new IsNotNull(0, 2);
@@ -309,7 +310,7 @@ public class TestVectorLogicalExpressions {
   }
 
   @Test
-  public void testBooleanFiltersOnColumns() {
+  public void testBooleanFiltersOnColumns() throws HiveException {
     VectorizedRowBatch batch = getBatchThreeBooleanCols();
 
     SelectColumnIsTrue expr = new SelectColumnIsTrue(0);
@@ -329,7 +330,7 @@ public class TestVectorLogicalExpressions {
   }
 
   @Test
-  public void testSelectColumnIsNull() {
+  public void testSelectColumnIsNull() throws HiveException {
     // has nulls, not repeating
     VectorizedRowBatch batch = getBatchThreeBooleanCols();
     SelectColumnIsNull expr = new SelectColumnIsNull(0);
@@ -362,7 +363,7 @@ public class TestVectorLogicalExpressions {
   }
 
   @Test
-  public void testSelectColumnIsNotNull() {
+  public void testSelectColumnIsNotNull() throws HiveException {
     // has nulls, not repeating
     VectorizedRowBatch batch = getBatchThreeBooleanCols();
     SelectColumnIsNotNull expr = new SelectColumnIsNotNull(0);
@@ -413,7 +414,7 @@ public class TestVectorLogicalExpressions {
     }
 
     @Override
-    public void evaluate(VectorizedRowBatch batch) {
+    public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
       if (childExpressions != null) {
         super.evaluateChildren(batch);
@@ -450,7 +451,7 @@ public class TestVectorLogicalExpressions {
     }
 
     @Override
-    public void evaluate(VectorizedRowBatch batch) {
+    public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
       if (childExpressions != null) {
         super.evaluateChildren(batch);
@@ -487,7 +488,7 @@ public class TestVectorLogicalExpressions {
     }
 
     @Override
-    public void evaluate(VectorizedRowBatch batch) {
+    public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
       if (childExpressions != null) {
         super.evaluateChildren(batch);
@@ -526,7 +527,7 @@ public class TestVectorLogicalExpressions {
     }
 
     @Override
-    public void evaluate(VectorizedRowBatch batch) {
+    public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
       if (childExpressions != null) {
         super.evaluateChildren(batch);
@@ -550,7 +551,7 @@ public class TestVectorLogicalExpressions {
   }
 
   @Test
-  public void testFilterExprOrExpr() {
+  public void testFilterExprOrExpr() throws HiveException {
     VectorizedRowBatch batch1 = getBatchThreeBooleanCols();
     VectorizedRowBatch batch2 = getBatchThreeBooleanCols();
 
@@ -591,7 +592,7 @@ public class TestVectorLogicalExpressions {
   }
 
   @Test
-  public void testFilterExprMultiOrExpr() {
+  public void testFilterExprMultiOrExpr() throws HiveException {
 
     // Select all with the first expression and expect the other 2 children to not be invoked.
 
@@ -719,7 +720,7 @@ public class TestVectorLogicalExpressions {
   }
 
   @Test
-  public void testFilterExprOrExprWithBatchReuse() {
+  public void testFilterExprOrExprWithBatchReuse() throws HiveException {
     VectorizedRowBatch batch1 = getBatchThreeBooleanCols();
 
     SelectColumnIsTrue expr1 = new SelectColumnIsTrue(0);
@@ -753,7 +754,7 @@ public class TestVectorLogicalExpressions {
   }
 
   @Test
-  public void testFilterExprOrExprWithSelectInUse() {
+  public void testFilterExprOrExprWithSelectInUse() throws HiveException {
     VectorizedRowBatch batch1 = getBatchThreeBooleanCols();
 
     SelectColumnIsTrue expr1 = new SelectColumnIsTrue(0);
@@ -783,7 +784,7 @@ public class TestVectorLogicalExpressions {
   }
 
   @Test
-  public void testFilterExprAndExpr() {
+  public void testFilterExprAndExpr() throws HiveException {
     VectorizedRowBatch batch1 = getBatchThreeBooleanCols();
 
     SelectColumnIsTrue expr1 = new SelectColumnIsTrue(0);
@@ -800,7 +801,7 @@ public class TestVectorLogicalExpressions {
   }
 
   @Test
-  public void testLongInExpr() {
+  public void testLongInExpr() throws HiveException {
 
     // check basic operation
     VectorizedRowBatch b = TestVectorMathFunctions.getVectorizedRowBatchLongInLongOut();
@@ -831,7 +832,7 @@ public class TestVectorLogicalExpressions {
   }
 
   @Test
-  public void testDoubleInExpr() {
+  public void testDoubleInExpr() throws HiveException {
 
     // check basic operation
     VectorizedRowBatch b = TestVectorMathFunctions.getVectorizedRowBatchDoubleInLongOut();

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorMathFunctions.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorMathFunctions.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorMathFunctions.java
index a8f94e5..e81844c 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorMathFunctions.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorMathFunctions.java
@@ -69,7 +69,7 @@ public class TestVectorMathFunctions {
   }
 
   @Test
-  public void testVectorRound() {
+  public void testVectorRound() throws HiveException {
     VectorizedRowBatch b = getVectorizedRowBatchDoubleInDoubleOut();
     VectorExpression expr = new FuncRoundDoubleToDouble(0, 1);
     DoubleColumnVector resultV = (DoubleColumnVector) b.cols[1];
@@ -106,7 +106,7 @@ public class TestVectorMathFunctions {
   }
 
   @Test
-  public void testRoundToDecimalPlaces() {
+  public void testRoundToDecimalPlaces() throws HiveException {
     VectorizedRowBatch b = getVectorizedRowBatchDoubleInDoubleOut();
     VectorExpression expr = new RoundWithNumDigitsDoubleToDouble(0, 4, 1);
     expr.evaluate(b);
@@ -357,7 +357,7 @@ public class TestVectorMathFunctions {
    * (for FuncRoundDoubleToDouble).
    */
   @Test
-  public void testVectorSin() {
+  public void testVectorSin() throws HiveException {
     VectorizedRowBatch b = getVectorizedRowBatchDoubleInDoubleOut();
     DoubleColumnVector resultV = (DoubleColumnVector) b.cols[1];
     b.cols[0].noNulls = true;
@@ -367,7 +367,7 @@ public class TestVectorMathFunctions {
   }
 
   @Test
-  public void testVectorCos() {
+  public void testVectorCos() throws HiveException {
     VectorizedRowBatch b = getVectorizedRowBatchDoubleInDoubleOut();
     DoubleColumnVector resultV = (DoubleColumnVector) b.cols[1];
     b.cols[0].noNulls = true;
@@ -377,7 +377,7 @@ public class TestVectorMathFunctions {
   }
 
   @Test
-  public void testVectorTan() {
+  public void testVectorTan() throws HiveException {
     VectorizedRowBatch b = getVectorizedRowBatchDoubleInDoubleOut();
     DoubleColumnVector resultV = (DoubleColumnVector) b.cols[1];
     b.cols[0].noNulls = true;
@@ -387,7 +387,7 @@ public class TestVectorMathFunctions {
   }
 
   @Test
-  public void testVectorASin() {
+  public void testVectorASin() throws HiveException {
     VectorizedRowBatch b = getVectorizedRowBatchDoubleInDoubleOut();
     DoubleColumnVector resultV = (DoubleColumnVector) b.cols[1];
     b.cols[0].noNulls = true;
@@ -397,7 +397,7 @@ public class TestVectorMathFunctions {
   }
 
   @Test
-  public void testVectorACos() {
+  public void testVectorACos() throws HiveException {
     VectorizedRowBatch b = getVectorizedRowBatchDoubleInDoubleOut();
     DoubleColumnVector resultV = (DoubleColumnVector) b.cols[1];
     b.cols[0].noNulls = true;
@@ -407,7 +407,7 @@ public class TestVectorMathFunctions {
   }
 
   @Test
-  public void testVectorATan() {
+  public void testVectorATan() throws HiveException {
     VectorizedRowBatch b = getVectorizedRowBatchDoubleInDoubleOut();
     DoubleColumnVector resultV = (DoubleColumnVector) b.cols[1];
     b.cols[0].noNulls = true;
@@ -417,7 +417,7 @@ public class TestVectorMathFunctions {
   }
 
   @Test
-  public void testVectorDegrees() {
+  public void testVectorDegrees() throws HiveException {
     VectorizedRowBatch b = getVectorizedRowBatchDoubleInDoubleOut();
     DoubleColumnVector resultV = (DoubleColumnVector) b.cols[1];
     b.cols[0].noNulls = true;
@@ -427,7 +427,7 @@ public class TestVectorMathFunctions {
   }
 
   @Test
-  public void testVectorRadians() {
+  public void testVectorRadians() throws HiveException {
     VectorizedRowBatch b = getVectorizedRowBatchDoubleInDoubleOut();
     DoubleColumnVector resultV = (DoubleColumnVector) b.cols[1];
     b.cols[0].noNulls = true;
@@ -437,7 +437,7 @@ public class TestVectorMathFunctions {
   }
 
   @Test
-  public void testVectorFloor() {
+  public void testVectorFloor() throws HiveException {
     VectorizedRowBatch b = getVectorizedRowBatchDoubleInLongOut();
     LongColumnVector resultV = (LongColumnVector) b.cols[1];
     b.cols[0].noNulls = true;
@@ -448,7 +448,7 @@ public class TestVectorMathFunctions {
   }
 
   @Test
-  public void testVectorCeil() {
+  public void testVectorCeil() throws HiveException {
     VectorizedRowBatch b = getVectorizedRowBatchDoubleInLongOut();
     LongColumnVector resultV = (LongColumnVector) b.cols[1];
     b.cols[0].noNulls = true;
@@ -459,7 +459,7 @@ public class TestVectorMathFunctions {
   }
 
   @Test
-  public void testVectorExp() {
+  public void testVectorExp() throws HiveException {
     VectorizedRowBatch b = getVectorizedRowBatchDoubleInDoubleOut();
     DoubleColumnVector resultV = (DoubleColumnVector) b.cols[1];
     b.cols[0].noNulls = true;
@@ -469,7 +469,7 @@ public class TestVectorMathFunctions {
   }
 
   @Test
-  public void testVectorLn() {
+  public void testVectorLn() throws HiveException {
 
     // test double->double version
     VectorizedRowBatch b = getVectorizedRowBatchDoubleInDoubleOut();
@@ -489,7 +489,7 @@ public class TestVectorMathFunctions {
   }
 
   @Test
-  public void testVectorLog2() {
+  public void testVectorLog2() throws HiveException {
 
     // test double->double version
     VectorizedRowBatch b = getVectorizedRowBatchDoubleInDoubleOut();
@@ -509,7 +509,7 @@ public class TestVectorMathFunctions {
   }
 
   @Test
-  public void testVectorLog10() {
+  public void testVectorLog10() throws HiveException {
 
     // test double->double version
     VectorizedRowBatch b = getVectorizedRowBatchDoubleInDoubleOut();
@@ -529,7 +529,7 @@ public class TestVectorMathFunctions {
   }
 
   @Test
-  public void testVectorRand() {
+  public void testVectorRand() throws HiveException {
     VectorizedRowBatch b = new VectorizedRowBatch(1);
     DoubleColumnVector v = new DoubleColumnVector();
     b.cols[0] = v;
@@ -565,7 +565,7 @@ public class TestVectorMathFunctions {
   }
 
   @Test
-  public void testVectorLogBase() {
+  public void testVectorLogBase() throws HiveException {
 
     // test double->double version
     VectorizedRowBatch b = getVectorizedRowBatchDoubleInDoubleOut();
@@ -577,7 +577,7 @@ public class TestVectorMathFunctions {
   }
 
   @Test
-  public void testVectorPosMod() {
+  public void testVectorPosMod() throws HiveException {
 
     // test double->double version
     VectorizedRowBatch b = getVectorizedRowBatchDoubleInDoubleOut();
@@ -620,7 +620,7 @@ public class TestVectorMathFunctions {
   }
 
   @Test
-  public void testVectorPosModWithFloatOutputType() {
+  public void testVectorPosModWithFloatOutputType() throws HiveException {
 
     // test double->double version
     VectorizedRowBatch b = getVectorizedRowBatchDoubleInDoubleOut();
@@ -644,7 +644,7 @@ public class TestVectorMathFunctions {
   }
 
   @Test
-  public void testVectorPower() {
+  public void testVectorPower() throws HiveException {
     VectorizedRowBatch b = getVectorizedRowBatchDoubleInDoubleOut();
     DoubleColumnVector resultV = (DoubleColumnVector) b.cols[1];
     b.cols[0].noNulls = true;
@@ -654,7 +654,7 @@ public class TestVectorMathFunctions {
   }
 
   @Test
-  public void testVectorSqrt() {
+  public void testVectorSqrt() throws HiveException {
     VectorizedRowBatch b = getVectorizedRowBatchDoubleInDoubleOut();
     DoubleColumnVector resultV = (DoubleColumnVector) b.cols[1];
     b.cols[0].noNulls = true;
@@ -664,7 +664,7 @@ public class TestVectorMathFunctions {
   }
 
   @Test
-  public void testVectorAbs() {
+  public void testVectorAbs() throws HiveException {
 
     // test double->double version
     VectorizedRowBatch b = getVectorizedRowBatchDoubleInDoubleOut();
@@ -686,7 +686,7 @@ public class TestVectorMathFunctions {
   }
 
   @Test
-  public void testVectorSign() {
+  public void testVectorSign() throws HiveException {
 
     // test double->double version
     VectorizedRowBatch b = getVectorizedRowBatchDoubleInDoubleOut();

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorScalarColArithmetic.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorScalarColArithmetic.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorScalarColArithmetic.java
index d93aa3b..a246a45 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorScalarColArithmetic.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorScalarColArithmetic.java
@@ -28,6 +28,7 @@ import org.apache.hadoop.hive.ql.exec.vector.TestVectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.LongScalarModuloLongColumn;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.LongScalarSubtractLongColumn;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.junit.Test;
 
 /**
@@ -67,7 +68,7 @@ public class TestVectorScalarColArithmetic {
   }
 
   @Test
-  public void testLongScalarModuloLongColNoNulls()  {
+  public void testLongScalarModuloLongColNoNulls() throws HiveException {
     VectorizedRowBatch batch = getBatchSingleLongVectorPositiveNonZero();
     LongScalarModuloLongColumn expr = new LongScalarModuloLongColumn(100, 0, 1);
     expr.evaluate(batch);
@@ -81,7 +82,7 @@ public class TestVectorScalarColArithmetic {
   }
 
   @Test
-  public void testLongScalarSubtractLongColNoNulls()  {
+  public void testLongScalarSubtractLongColNoNulls() throws HiveException {
     VectorizedRowBatch batch = getVectorizedRowBatchSingleLongVector(
         VectorizedRowBatch.DEFAULT_SIZE);
     LongScalarSubtractLongColumn expr = new LongScalarSubtractLongColumn(100, 0, 1);
@@ -96,7 +97,7 @@ public class TestVectorScalarColArithmetic {
   }
 
   @Test
-  public void testLongScalarSubtractLongColWithNulls()  {
+  public void testLongScalarSubtractLongColWithNulls() throws HiveException {
     VectorizedRowBatch batch = getVectorizedRowBatchSingleLongVector(
         VectorizedRowBatch.DEFAULT_SIZE);
     LongColumnVector lcv = (LongColumnVector) batch.cols[0];
@@ -119,7 +120,7 @@ public class TestVectorScalarColArithmetic {
   }
 
   @Test
-  public void testLongScalarSubtractLongColWithRepeating() {
+  public void testLongScalarSubtractLongColWithRepeating() throws HiveException {
     LongColumnVector in, out;
     VectorizedRowBatch batch;
     LongScalarSubtractLongColumn expr;
@@ -167,7 +168,7 @@ public class TestVectorScalarColArithmetic {
   }
 
   @Test
-  public void testLongScalarDivide() {
+  public void testLongScalarDivide() throws HiveException {
     VectorizedRowBatch batch =
         TestVectorArithmeticExpressions.getVectorizedRowBatch2LongInDoubleOut();
     LongColDivideLongScalar expr = new LongColDivideLongScalar(0, 100, 2);
@@ -191,7 +192,7 @@ public class TestVectorScalarColArithmetic {
   }
 
   @Test
-  public void testScalarLongDivide() {
+  public void testScalarLongDivide() throws HiveException {
     VectorizedRowBatch batch =
         TestVectorArithmeticExpressions.getVectorizedRowBatch2LongInDoubleOut();
     LongScalarDivideLongColumn expr = new LongScalarDivideLongColumn(100, 0, 2);
@@ -219,7 +220,7 @@ public class TestVectorScalarColArithmetic {
   }
 
   @Test
-  public void testBooleanValuedLongIn() {
+  public void testBooleanValuedLongIn() throws HiveException {
     VectorizedRowBatch batch = getBatch();
     long[] a = new long[2];
     a[0] = 20;

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorStringExpressions.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorStringExpressions.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorStringExpressions.java
index 202f18c..80dddcb 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorStringExpressions.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorStringExpressions.java
@@ -3222,7 +3222,7 @@ public class TestVectorStringExpressions {
 
   @Test
   // Test string column to string literal comparison
-  public void testStringColCompareStringScalarFilter() {
+  public void testStringColCompareStringScalarFilter() throws HiveException {
     VectorizedRowBatch batch = makeStringBatch();
     VectorExpression expr;
     expr = new FilterStringGroupColEqualStringScalar(0, red2);
@@ -3252,7 +3252,7 @@ public class TestVectorStringExpressions {
 
   @Test
   // Test string column to CHAR literal comparison
-  public void testStringColCompareCharScalarFilter() {
+  public void testStringColCompareCharScalarFilter() throws HiveException {
     VectorizedRowBatch batch = makeStringBatch();
     VectorExpression expr;
     expr = new FilterStringGroupColEqualCharScalar(0, new HiveChar(new String(red2), 10));
@@ -3282,7 +3282,7 @@ public class TestVectorStringExpressions {
 
   @Test
   // Test string column to VARCHAR literal comparison
-  public void testStringColCompareVarCharScalarFilter() {
+  public void testStringColCompareVarCharScalarFilter() throws HiveException {
     VectorizedRowBatch batch = makeStringBatch();
     VectorExpression expr;
     expr = new FilterStringGroupColEqualVarCharScalar(0, new HiveVarchar(new String(red2), 10));
@@ -3311,7 +3311,7 @@ public class TestVectorStringExpressions {
   }
 
   @Test
-  public void testStringColCompareStringScalarProjection() {
+  public void testStringColCompareStringScalarProjection() throws HiveException {
     VectorizedRowBatch batch = makeStringBatch();
     VectorExpression expr;
 
@@ -3334,7 +3334,7 @@ public class TestVectorStringExpressions {
   }
 
   @Test
-  public void testStringColCompareCharScalarProjection() {
+  public void testStringColCompareCharScalarProjection() throws HiveException {
     VectorizedRowBatch batch = makeStringBatch();
     VectorExpression expr;
 
@@ -3357,7 +3357,7 @@ public class TestVectorStringExpressions {
   }
 
   @Test
-  public void testStringColCompareVarCharScalarProjection() {
+  public void testStringColCompareVarCharScalarProjection() throws HiveException {
     VectorizedRowBatch batch = makeStringBatch();
     VectorExpression expr;
 
@@ -3381,7 +3381,7 @@ public class TestVectorStringExpressions {
 
   @Test
   // Test string literal to string column comparison
-  public void testStringScalarCompareStringCol() {
+  public void testStringScalarCompareStringCol() throws HiveException {
     VectorizedRowBatch batch = makeStringBatch();
     VectorExpression expr;
     expr = new FilterStringScalarEqualStringGroupColumn(red2, 0);
@@ -3411,7 +3411,7 @@ public class TestVectorStringExpressions {
 
   @Test
   // Test CHAR literal to string column comparison
-  public void testCharScalarCompareStringCol() {
+  public void testCharScalarCompareStringCol() throws HiveException {
     VectorizedRowBatch batch = makeStringBatch();
     VectorExpression expr;
     expr = new FilterCharScalarEqualStringGroupColumn(new HiveChar(new String(red2), 8), 0);
@@ -3441,7 +3441,7 @@ public class TestVectorStringExpressions {
 
   @Test
   // Test VARCHAR literal to string column comparison
-  public void testVarCharScalarCompareStringCol() {
+  public void testVarCharScalarCompareStringCol() throws HiveException {
     VectorizedRowBatch batch = makeStringBatch();
     VectorExpression expr;
     expr = new FilterVarCharScalarEqualStringGroupColumn(new HiveVarchar(new String(red2), 8), 0);
@@ -3470,7 +3470,7 @@ public class TestVectorStringExpressions {
   }
 
   @Test
-  public void testStringScalarCompareStringColProjection() {
+  public void testStringScalarCompareStringColProjection() throws HiveException {
     VectorizedRowBatch batch = makeStringBatch();
     VectorExpression expr;
 
@@ -3493,7 +3493,7 @@ public class TestVectorStringExpressions {
   }
 
   @Test
-  public void testCharScalarCompareStringColProjection() {
+  public void testCharScalarCompareStringColProjection() throws HiveException {
     VectorizedRowBatch batch = makeStringBatch();
     VectorExpression expr;
 
@@ -3516,7 +3516,7 @@ public class TestVectorStringExpressions {
   }
 
   @Test
-  public void testVarCharScalarCompareStringColProjection() {
+  public void testVarCharScalarCompareStringColProjection() throws HiveException {
     VectorizedRowBatch batch = makeStringBatch();
     VectorExpression expr;
 
@@ -3538,7 +3538,7 @@ public class TestVectorStringExpressions {
     Assert.assertEquals(0, outVector.vector[2]);
   }
   @Test
-  public void testStringColCompareStringColFilter() {
+  public void testStringColCompareStringColFilter() throws HiveException {
     VectorizedRowBatch batch;
     VectorExpression expr;
 
@@ -3690,7 +3690,7 @@ public class TestVectorStringExpressions {
   }
 
   @Test
-  public void testStringColCompareStringColProjection() {
+  public void testStringColCompareStringColProjection() throws HiveException {
     VectorizedRowBatch batch;
     VectorExpression expr;
     long [] outVector;
@@ -4029,7 +4029,7 @@ public class TestVectorStringExpressions {
   }
 
   @Test
-  public void testColLower() {
+  public void testColLower() throws HiveException {
     // has nulls, not repeating
     VectorizedRowBatch batch = makeStringBatchMixedCase();
     StringLower expr = new StringLower(0, 1);
@@ -4077,7 +4077,7 @@ public class TestVectorStringExpressions {
   }
 
   @Test
-  public void testColUpper() {
+  public void testColUpper() throws HiveException {
 
     // no nulls, not repeating
 
@@ -4096,7 +4096,7 @@ public class TestVectorStringExpressions {
   }
 
   @Test
-  public void testStringLength() {
+  public void testStringLength() throws HiveException {
 
     // has nulls, not repeating
     VectorizedRowBatch batch = makeStringBatchMixedCharSize();
@@ -4435,7 +4435,7 @@ public class TestVectorStringExpressions {
   }
 
   @Test
-  public void testColConcatStringScalar() {
+  public void testColConcatStringScalar() throws HiveException {
 
     // has nulls, not repeating
     VectorizedRowBatch batch = makeStringBatch();
@@ -4497,7 +4497,7 @@ public class TestVectorStringExpressions {
   }
 
   @Test
-  public void testColConcatCharScalar() {
+  public void testColConcatCharScalar() throws HiveException {
 
     // has nulls, not repeating
     VectorizedRowBatch batch = makeStringBatch();
@@ -4559,7 +4559,7 @@ public class TestVectorStringExpressions {
   }
 
   @Test
-  public void testColConcatVarCharScalar() {
+  public void testColConcatVarCharScalar() throws HiveException {
 
     // has nulls, not repeating
     VectorizedRowBatch batch = makeStringBatch();
@@ -4621,7 +4621,7 @@ public class TestVectorStringExpressions {
   }
 
   @Test
-  public void testStringScalarConcatCol() {
+  public void testStringScalarConcatCol() throws HiveException {
 
     // has nulls, not repeating
     VectorizedRowBatch batch = makeStringBatch();
@@ -4683,7 +4683,7 @@ public class TestVectorStringExpressions {
   }
 
   @Test
-  public void testCharScalarConcatCol() {
+  public void testCharScalarConcatCol() throws HiveException {
 
     // has nulls, not repeating
     VectorizedRowBatch batch = makeStringBatch();
@@ -4745,7 +4745,7 @@ public class TestVectorStringExpressions {
   }
 
   @Test
-  public void testVarCharScalarConcatCol() {
+  public void testVarCharScalarConcatCol() throws HiveException {
 
     // has nulls, not repeating
     VectorizedRowBatch batch = makeStringBatch();
@@ -4807,7 +4807,7 @@ public class TestVectorStringExpressions {
   }
 
   @Test
-  public void testColConcatCol() {
+  public void testColConcatCol() throws HiveException {
 
     // has nulls, not repeating
     VectorizedRowBatch batch = makeStringBatch2In1Out();
@@ -4949,7 +4949,7 @@ public class TestVectorStringExpressions {
   }
 
   @Test
-  public void testSubstrStart() throws UnsupportedEncodingException {
+  public void testSubstrStart() throws HiveException, UnsupportedEncodingException {
     // Testing no nulls and no repeating
     VectorizedRowBatch batch = new VectorizedRowBatch(2);
     BytesColumnVector v = new BytesColumnVector();
@@ -5148,7 +5148,7 @@ public class TestVectorStringExpressions {
   }
 
   @Test
-  public void testSubstrStartLen() throws UnsupportedEncodingException {
+  public void testSubstrStartLen() throws HiveException, UnsupportedEncodingException {
     // Testing no nulls and no repeating
 
     VectorizedRowBatch batch = new VectorizedRowBatch(2);
@@ -5403,7 +5403,7 @@ public class TestVectorStringExpressions {
   }
 
   @Test
-  public void testVectorLTrim() {
+  public void testVectorLTrim() throws HiveException {
     VectorizedRowBatch b = makeTrimBatch();
     VectorExpression expr = new StringLTrim(0, 1);
     expr.evaluate(b);
@@ -5423,7 +5423,7 @@ public class TestVectorStringExpressions {
   }
 
   @Test
-  public void testVectorRTrim() {
+  public void testVectorRTrim() throws HiveException {
     VectorizedRowBatch b = makeTrimBatch();
     VectorExpression expr = new StringRTrim(0, 1);
     expr.evaluate(b);
@@ -5443,7 +5443,7 @@ public class TestVectorStringExpressions {
   }
 
   @Test
-  public void testVectorTrim() {
+  public void testVectorTrim() throws HiveException {
     VectorizedRowBatch b = makeTrimBatch();
     VectorExpression expr = new StringTrim(0, 1);
     expr.evaluate(b);
@@ -5481,7 +5481,7 @@ public class TestVectorStringExpressions {
 
   // Test boolean-valued (non-filter) IN expression for strings
   @Test
-  public void testStringInExpr() {
+  public void testStringInExpr() throws HiveException {
 
     // test basic operation
     VectorizedRowBatch b = makeStringBatch();

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorTypeCasts.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorTypeCasts.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorTypeCasts.java
index 6aa6da9..3f17d4c 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorTypeCasts.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorTypeCasts.java
@@ -58,7 +58,7 @@ import org.junit.Test;
 public class TestVectorTypeCasts {
 
   @Test
-  public void testVectorCastLongToDouble() {
+  public void testVectorCastLongToDouble() throws HiveException {
     VectorizedRowBatch b = TestVectorMathFunctions.getVectorizedRowBatchLongInDoubleOut();
     DoubleColumnVector resultV = (DoubleColumnVector) b.cols[1];
     b.cols[0].noNulls = true;
@@ -68,7 +68,7 @@ public class TestVectorTypeCasts {
   }
 
   @Test
-  public void testVectorCastDoubleToLong() {
+  public void testVectorCastDoubleToLong() throws HiveException {
     VectorizedRowBatch b = TestVectorMathFunctions.getVectorizedRowBatchDoubleInLongOut();
     LongColumnVector resultV = (LongColumnVector) b.cols[1];
     b.cols[0].noNulls = true;
@@ -78,7 +78,7 @@ public class TestVectorTypeCasts {
   }
 
   @Test
-  public void testCastDateToTimestamp() {
+  public void testCastDateToTimestamp() throws HiveException {
     int[] intValues = new int[500];
     VectorizedRowBatch b = TestVectorMathFunctions.getVectorizedRowBatchDateInTimestampOut(intValues);
     TimestampColumnVector resultV = (TimestampColumnVector) b.cols[1];
@@ -93,7 +93,7 @@ public class TestVectorTypeCasts {
   }
 
   @Test
-  public void testCastDoubleToBoolean() {
+  public void testCastDoubleToBoolean() throws HiveException {
     VectorizedRowBatch b = TestVectorMathFunctions.getVectorizedRowBatchDoubleInLongOut();
     LongColumnVector resultV = (LongColumnVector) b.cols[1];
     b.cols[0].noNulls = true;
@@ -104,7 +104,7 @@ public class TestVectorTypeCasts {
   }
 
   @Test
-  public void testCastDoubleToTimestamp() {
+  public void testCastDoubleToTimestamp() throws HiveException {
     VectorizedRowBatch b = TestVectorMathFunctions.getVectorizedRowBatchDoubleInTimestampOut();
     TimestampColumnVector resultV = (TimestampColumnVector) b.cols[1];
     b.cols[0].noNulls = true;
@@ -115,7 +115,7 @@ public class TestVectorTypeCasts {
   }
 
   @Test
-  public void testCastLongToBoolean() {
+  public void testCastLongToBoolean() throws HiveException {
     VectorizedRowBatch b = TestVectorMathFunctions.getVectorizedRowBatchLongInLongOut();
     LongColumnVector inV = (LongColumnVector) b.cols[0];
     inV.vector[0] = 0;  // make one entry produce false in result
@@ -128,7 +128,7 @@ public class TestVectorTypeCasts {
   }
 
   @Test
-  public void testCastStringToBoolean() {
+  public void testCastStringToBoolean() throws HiveException {
     VectorizedRowBatch b = TestVectorMathFunctions.getVectorizedRowBatchStringInLongOut();
     LongColumnVector resultV = (LongColumnVector) b.cols[1];
     b.cols[0].noNulls = true;
@@ -145,7 +145,7 @@ public class TestVectorTypeCasts {
   }
 
   @Test
-  public void testCastLongToTimestamp() {
+  public void testCastLongToTimestamp() throws HiveException {
     long[] longValues = new long[500];
     VectorizedRowBatch b = TestVectorMathFunctions.getVectorizedRowBatchLongInTimestampOut(longValues);
     TimestampColumnVector resultV = (TimestampColumnVector) b.cols[1];
@@ -160,7 +160,7 @@ public class TestVectorTypeCasts {
   }
 
   @Test
-  public void testCastTimestampToLong() {
+  public void testCastTimestampToLong() throws HiveException {
     long[] longValues = new long[500];
     VectorizedRowBatch b = TestVectorMathFunctions.getVectorizedRowBatchTimestampInLongOut(longValues);
     TimestampColumnVector inV = (TimestampColumnVector) b.cols[0];
@@ -178,7 +178,7 @@ public class TestVectorTypeCasts {
   }
 
   @Test
-  public void testCastTimestampToDouble() {
+  public void testCastTimestampToDouble() throws HiveException {
     double[] doubleValues = new double[500];
     VectorizedRowBatch b = TestVectorMathFunctions.getVectorizedRowBatchTimestampInDoubleOut(doubleValues);
     TimestampColumnVector inV = (TimestampColumnVector) b.cols[0];
@@ -219,7 +219,7 @@ public class TestVectorTypeCasts {
   }
 
   @Test
-  public void testCastBooleanToString() {
+  public void testCastBooleanToString() throws HiveException {
     byte[] t = toBytes("TRUE");
     byte[] f = toBytes("FALSE");
     VectorizedRowBatch b = TestVectorMathFunctions.getBatchForStringMath();
@@ -319,7 +319,7 @@ public class TestVectorTypeCasts {
   }
 
   @Test
-  public void testCastDecimalToDouble() {
+  public void testCastDecimalToDouble() throws HiveException {
 
     final double eps = 0.000001d; // tolerance to check double equality
 
@@ -422,7 +422,7 @@ public class TestVectorTypeCasts {
   }
 
   @Test
-  public void testCastDecimalToTimestamp() {
+  public void testCastDecimalToTimestamp() throws HiveException {
     double[] doubleValues = new double[500];
     VectorizedRowBatch b = getBatchDecimalTimestamp(doubleValues);
     VectorExpression expr = new CastDecimalToTimestamp(0, 1);
@@ -477,7 +477,7 @@ public class TestVectorTypeCasts {
   }
 
   @Test
-  public void testCastLongToDecimal() {
+  public void testCastLongToDecimal() throws HiveException {
     VectorizedRowBatch b = getBatchLongDecimal();
     VectorExpression expr = new CastLongToDecimal(0, 1);
     expr.evaluate(b);
@@ -540,7 +540,7 @@ public class TestVectorTypeCasts {
   }
 
   @Test
-  public void testCastDoubleToDecimal() {
+  public void testCastDoubleToDecimal() throws HiveException {
     VectorizedRowBatch b = getBatchDoubleDecimal();
     VectorExpression expr = new CastDoubleToDecimal(0, 1);
     expr.evaluate(b);
@@ -577,7 +577,7 @@ public class TestVectorTypeCasts {
   }
 
   @Test
-  public void testCastStringToDecimal() {
+  public void testCastStringToDecimal() throws HiveException {
     VectorizedRowBatch b = getBatchStringDecimal();
     VectorExpression expr = new CastStringToDecimal(0, 1);
     expr.evaluate(b);
@@ -607,7 +607,7 @@ public class TestVectorTypeCasts {
   }
 
   @Test
-  public void testCastTimestampToDecimal() {
+  public void testCastTimestampToDecimal() throws HiveException {
 
     // The input timestamps are stored as long values
     // measured in nanoseconds from the epoch.

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/udf/TestVectorUDFAdaptor.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/udf/TestVectorUDFAdaptor.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/udf/TestVectorUDFAdaptor.java
index 7e512ba..46834d8 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/udf/TestVectorUDFAdaptor.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/udf/TestVectorUDFAdaptor.java
@@ -61,7 +61,7 @@ public class TestVectorUDFAdaptor {
   }
 
   @Test
-  public void testLongUDF()  {
+  public void testLongUDF() throws HiveException {
 
     // create a syntax tree for a simple function call "longudf(col0)"
     ExprNodeGenericFuncDesc funcDesc;
@@ -124,7 +124,7 @@ public class TestVectorUDFAdaptor {
   }
 
   @Test
-  public void testMultiArgumentUDF() {
+  public void testMultiArgumentUDF() throws HiveException {
 
     // create a syntax tree for a function call "testudf(col0, col1, col2)"
     ExprNodeGenericFuncDesc funcDesc;
@@ -246,7 +246,7 @@ public class TestVectorUDFAdaptor {
 
   // test the UDF adaptor for a generic UDF (as opposed to a legacy UDF)
   @Test
-  public void testGenericUDF() {
+  public void testGenericUDF() throws HiveException {
 
     // create a syntax tree for a function call 'myisnull(col0, "UNKNOWN")'
     ExprNodeGenericFuncDesc funcDesc;

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/test/queries/clientpositive/vector_retry_failure.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/vector_retry_failure.q b/ql/src/test/queries/clientpositive/vector_retry_failure.q
new file mode 100644
index 0000000..448af73
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/vector_retry_failure.q
@@ -0,0 +1,15 @@
+--! qt:dataset:src
+
+SET hive.vectorized.execution.enabled=true;
+create table tx(a int,f string);
+insert into tx values (1,'non_existent_file');
+
+set zzz=1;
+set reexec.overlay.zzz=2;
+
+set hive.query.reexecution.enabled=true;
+set hive.query.reexecution.strategies=overlay;
+
+explain vectorization expression
+select assert_true(${hiveconf:zzz} > a) from tx group by a;
+select assert_true(${hiveconf:zzz} > a) from tx group by a;

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/test/results/clientpositive/llap/vector_retry_failure.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/vector_retry_failure.q.out b/ql/src/test/results/clientpositive/llap/vector_retry_failure.q.out
new file mode 100644
index 0000000..7bb21a4
--- /dev/null
+++ b/ql/src/test/results/clientpositive/llap/vector_retry_failure.q.out
@@ -0,0 +1,160 @@
+PREHOOK: query: create table tx(a int,f string)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@tx
+POSTHOOK: query: create table tx(a int,f string)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@tx
+PREHOOK: query: insert into tx values (1,'non_existent_file')
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@tx
+POSTHOOK: query: insert into tx values (1,'non_existent_file')
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@tx
+POSTHOOK: Lineage: tx.a SCRIPT []
+POSTHOOK: Lineage: tx.f SCRIPT []
+PREHOOK: query: explain vectorization expression
+select assert_true(1 > a) from tx group by a
+PREHOOK: type: QUERY
+POSTHOOK: query: explain vectorization expression
+select assert_true(1 > a) from tx group by a
+POSTHOOK: type: QUERY
+PLAN VECTORIZATION:
+  enabled: true
+  enabledConditionsMet: [hive.vectorized.execution.enabled IS true]
+
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Edges:
+        Reducer 2 <- Map 1 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: tx
+                  Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+                  TableScan Vectorization:
+                      native: true
+                  Select Operator
+                    expressions: a (type: int)
+                    outputColumnNames: a
+                    Select Vectorization:
+                        className: VectorSelectOperator
+                        native: true
+                        projectedOutputColumnNums: [0]
+                    Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+                    Group By Operator
+                      Group By Vectorization:
+                          className: VectorGroupByOperator
+                          groupByMode: HASH
+                          keyExpressions: col 0:int
+                          native: false
+                          vectorProcessingMode: HASH
+                          projectedOutputColumnNums: []
+                      keys: a (type: int)
+                      mode: hash
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
+                        Reduce Sink Vectorization:
+                            className: VectorReduceSinkLongOperator
+                            native: true
+                            nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true
+                        Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+            Execution mode: vectorized, llap
+            LLAP IO: no inputs
+            Map Vectorization:
+                enabled: true
+                enabledConditionsMet: hive.vectorized.use.vector.serde.deserialize IS true
+                inputFormatFeatureSupport: [DECIMAL_64]
+                vectorizationSupportRemovedReasons: [DECIMAL_64 disabled because LLAP is enabled]
+                featureSupportInUse: []
+                inputFileFormats: org.apache.hadoop.mapred.TextInputFormat
+                allNative: false
+                usesVectorUDFAdaptor: false
+                vectorized: true
+        Reducer 2 
+            Execution mode: vectorized, llap
+            Reduce Vectorization:
+                enabled: true
+                enableConditionsMet: hive.vectorized.execution.reduce.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true
+                allNative: false
+                usesVectorUDFAdaptor: true
+                vectorized: true
+            Reduce Operator Tree:
+              Group By Operator
+                Group By Vectorization:
+                    className: VectorGroupByOperator
+                    groupByMode: MERGEPARTIAL
+                    keyExpressions: col 0:int
+                    native: false
+                    vectorProcessingMode: MERGE_PARTIAL
+                    projectedOutputColumnNums: []
+                keys: KEY._col0 (type: int)
+                mode: mergepartial
+                outputColumnNames: _col0
+                Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+                Select Operator
+                  expressions: assert_true((_col0 < 1)) (type: void)
+                  outputColumnNames: _col0
+                  Select Vectorization:
+                      className: VectorSelectOperator
+                      native: true
+                      projectedOutputColumnNums: [2]
+                      selectExpressions: VectorUDFAdaptor(assert_true((_col0 < 1)))(children: LongColLessLongScalar(col 0:int, val 1) -> 1:boolean) -> 2:void
+                  Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+                  File Output Operator
+                    compressed: false
+                    File Sink Vectorization:
+                        className: VectorFileSinkOperator
+                        native: false
+                    Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+                    table:
+                        input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                        output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                        serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: select assert_true(1 > a) from tx group by a
+PREHOOK: type: QUERY
+PREHOOK: Input: default@tx
+#### A masked pattern was here ####
+Status: Failed
+Vertex failed, vertexName=Reducer 2, vertexId=vertex_#ID#, diagnostics=[Task failed, taskId=task_#ID#, diagnostics=[TaskAttempt 0 failed, info=[Error: Error while running task ( failure ) : attempt_#ID#:java.lang.RuntimeException: java.lang.RuntimeException: Hive Runtime Error while closing operators: ASSERT_TRUE(): assertion failed.
+#### A masked pattern was here ####
+], TaskAttempt 1 failed, info=[Error: Error while running task ( failure ) : attempt_#ID#:java.lang.RuntimeException: java.lang.RuntimeException: Hive Runtime Error while closing operators: ASSERT_TRUE(): assertion failed.
+#### A masked pattern was here ####
+]], Vertex did not succeed due to OWN_TASK_FAILURE, failedTasks:1 killedTasks:0, Vertex vertex_#ID# [Reducer 2] killed/failed due to:OWN_TASK_FAILURE]
+DAG did not succeed due to VERTEX_FAILURE. failedVertices:1 killedVertices:0
+FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.tez.TezTask. Vertex failed, vertexName=Reducer 2, vertexId=vertex_#ID#, diagnostics=[Task failed, taskId=task_#ID#, diagnostics=[TaskAttempt 0 failed, info=[Error: Error while running task ( failure ) : attempt_#ID#:java.lang.RuntimeException: java.lang.RuntimeException: Hive Runtime Error while closing operators: ASSERT_TRUE(): assertion failed.
+#### A masked pattern was here ####
+], TaskAttempt 1 failed, info=[Error: Error while running task ( failure ) : attempt_#ID#:java.lang.RuntimeException: java.lang.RuntimeException: Hive Runtime Error while closing operators: ASSERT_TRUE(): assertion failed.
+#### A masked pattern was here ####
+]], Vertex did not succeed due to OWN_TASK_FAILURE, failedTasks:1 killedTasks:0, Vertex vertex_#ID# [Reducer 2] killed/failed due to:OWN_TASK_FAILURE]DAG did not succeed due to VERTEX_FAILURE. failedVertices:1 killedVertices:0
+PREHOOK: query: select assert_true(2 > a) from tx group by a
+PREHOOK: type: QUERY
+PREHOOK: Input: default@tx
+#### A masked pattern was here ####
+POSTHOOK: query: select assert_true(2 > a) from tx group by a
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@tx
+#### A masked pattern was here ####
+NULL


[17/50] [abbrv] hive git commit: HIVE-19472: HiveStreamingConnection swallows exception on partition creation (Prasanth Jayachandran reviewed by Sergey Shelukhin)

Posted by vg...@apache.org.
HIVE-19472: HiveStreamingConnection swallows exception on partition creation (Prasanth Jayachandran reviewed by Sergey Shelukhin)


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

Branch: refs/heads/branch-3.0.0
Commit: c03ae0b3d89226a7a00124339b763aedcbe1e22f
Parents: d2c9700
Author: Prasanth Jayachandran <pr...@apache.org>
Authored: Thu May 10 11:54:54 2018 -0700
Committer: Prasanth Jayachandran <pr...@apache.org>
Committed: Thu May 10 11:59:28 2018 -0700

----------------------------------------------------------------------
 .../java/org/apache/hive/streaming/HiveStreamingConnection.java  | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/c03ae0b3/streaming/src/java/org/apache/hive/streaming/HiveStreamingConnection.java
----------------------------------------------------------------------
diff --git a/streaming/src/java/org/apache/hive/streaming/HiveStreamingConnection.java b/streaming/src/java/org/apache/hive/streaming/HiveStreamingConnection.java
index 85887b2..f697211 100644
--- a/streaming/src/java/org/apache/hive/streaming/HiveStreamingConnection.java
+++ b/streaming/src/java/org/apache/hive/streaming/HiveStreamingConnection.java
@@ -373,7 +373,7 @@ public class HiveStreamingConnection implements StreamingConnection {
       exists = true;
     } catch (HiveException | TException e) {
       throw new StreamingException("Unable to creation partition for values: " + partitionValues + " connection: " +
-        toConnectionInfoString());
+        toConnectionInfoString(), e);
     }
     return new PartitionInfo(partName, partLocation, exists);
   }
@@ -460,7 +460,7 @@ public class HiveStreamingConnection implements StreamingConnection {
     }
 
     if (currentTransactionBatch.isClosed()) {
-      throw new IllegalStateException("Cannot begin next transaction on a closed streaming connection");
+      throw new StreamingException("Cannot begin next transaction on a closed streaming connection");
     }
 
     if (currentTransactionBatch.remainingTransactions() == 0) {


[10/50] [abbrv] hive git commit: Revert "Revert "HIVE-19310 : Metastore: MetaStoreDirectSql.ensureDbInit has some slow DN calls which might need to be run only in test env (Sergey Shelukhin, reviewed by Gopal Vijayaraghavan)""

Posted by vg...@apache.org.
Revert "Revert "HIVE-19310 : Metastore: MetaStoreDirectSql.ensureDbInit has some slow DN calls which might need to be run only in test env (Sergey Shelukhin, reviewed by Gopal Vijayaraghavan)""

This reverts commit 71d77ab1b268b200c93cb9e69f2f2ee74afa8c96.


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

Branch: refs/heads/branch-3.0.0
Commit: 92bc9cf04ef3779cea2959917a2063e16ce86fac
Parents: aa040c5
Author: sergey <se...@apache.org>
Authored: Tue May 8 17:20:19 2018 -0700
Committer: sergey <se...@apache.org>
Committed: Tue May 8 17:20:19 2018 -0700

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hive/metastore/MetaStoreDirectSql.java | 3 ++-
 .../test/java/org/apache/hadoop/hive/metastore/TestOldSchema.java | 1 +
 .../src/test/java/org/apache/hadoop/hive/metastore/TestStats.java | 1 +
 3 files changed, 4 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/92bc9cf0/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreDirectSql.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreDirectSql.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreDirectSql.java
index 997f5fd..e2ca6d2 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreDirectSql.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreDirectSql.java
@@ -179,7 +179,8 @@ class MetaStoreDirectSql {
           + " Disabling directSQL as it uses hand-hardcoded SQL with that assumption.");
       isCompatibleDatastore = false;
     } else {
-      isCompatibleDatastore = ensureDbInit() && runTestQuery();
+      boolean isInTest = MetastoreConf.getBoolVar(conf, ConfVars.HIVE_IN_TEST);
+      isCompatibleDatastore = (!isInTest || ensureDbInit()) && runTestQuery();
       if (isCompatibleDatastore) {
         LOG.info("Using direct SQL, underlying DB is " + dbType);
       }

http://git-wip-us.apache.org/repos/asf/hive/blob/92bc9cf0/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestOldSchema.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestOldSchema.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestOldSchema.java
index 49033d3..54b14c3 100644
--- a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestOldSchema.java
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestOldSchema.java
@@ -98,6 +98,7 @@ public class TestOldSchema {
   public void setUp() throws Exception {
     conf = MetastoreConf.newMetastoreConf();
     MetastoreConf.setBoolVar(conf, MetastoreConf.ConfVars.STATS_FETCH_BITVECTOR, false);
+    MetastoreConf.setBoolVar(conf, MetastoreConf.ConfVars.HIVE_IN_TEST, true);
     MetaStoreTestUtils.setConfForStandloneMode(conf);
 
     store = new ObjectStore();

http://git-wip-us.apache.org/repos/asf/hive/blob/92bc9cf0/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestStats.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestStats.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestStats.java
index 6cca062..4d88505 100644
--- a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestStats.java
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestStats.java
@@ -76,6 +76,7 @@ public class TestStats {
     conf = MetastoreConf.newMetastoreConf();
     MetaStoreTestUtils.setConfForStandloneMode(conf);
     MetastoreConf.setBoolVar(conf, MetastoreConf.ConfVars.AGGREGATE_STATS_CACHE_ENABLED, false);
+    MetastoreConf.setBoolVar(conf, MetastoreConf.ConfVars.HIVE_IN_TEST, true);
     // Get new client
     client = new HiveMetaStoreClient(conf);
   }


[36/50] [abbrv] hive git commit: HIVE-19466: Update constraint violation error message(Vineet Garg, reviewed by Jesus Camacho Rodriguez)

Posted by vg...@apache.org.
HIVE-19466: Update constraint violation error message(Vineet Garg, reviewed by Jesus Camacho Rodriguez)


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

Branch: refs/heads/branch-3.0.0
Commit: ba7155d33ba6598bfad99403af76296fc52483f4
Parents: 8c12a11
Author: Vineet Garg <vg...@apache.org>
Authored: Thu May 10 14:12:04 2018 -0700
Committer: Vineet Garg <vg...@apache.org>
Committed: Mon May 14 11:46:19 2018 -0700

----------------------------------------------------------------------
 .../hadoop/hive/ql/exec/FunctionRegistry.java   |  2 +-
 .../generic/GenericUDFEnforceConstraint.java    | 77 ++++++++++++++++++++
 .../GenericUDFEnforceNotNullConstraint.java     | 77 --------------------
 .../TestGenericUDFEnforceConstraint.java        | 75 +++++++++++++++++++
 .../TestGenericUDFEnforceNotNullConstraint.java | 75 -------------------
 .../alter_notnull_constraint_violation.q.out    |  2 +-
 .../insert_into_acid_notnull.q.out              |  2 +-
 .../insert_into_notnull_constraint.q.out        |  2 +-
 .../insert_multi_into_notnull.q.out             |  2 +-
 .../insert_overwrite_notnull_constraint.q.out   |  2 +-
 .../update_notnull_constraint.q.out             |  2 +-
 11 files changed, 159 insertions(+), 159 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/ba7155d3/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java
index 4611ce9..a1f549a 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java
@@ -214,7 +214,7 @@ public final class FunctionRegistry {
     system.registerUDF("rand", UDFRand.class, false);
     system.registerGenericUDF("abs", GenericUDFAbs.class);
     system.registerGenericUDF("sq_count_check", GenericUDFSQCountCheck.class);
-    system.registerGenericUDF("enforce_constraint", GenericUDFEnforceNotNullConstraint.class);
+    system.registerGenericUDF("enforce_constraint", GenericUDFEnforceConstraint.class);
     system.registerGenericUDF("pmod", GenericUDFPosMod.class);
 
     system.registerUDF("ln", UDFLn.class, false);

http://git-wip-us.apache.org/repos/asf/hive/blob/ba7155d3/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFEnforceConstraint.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFEnforceConstraint.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFEnforceConstraint.java
new file mode 100644
index 0000000..aa0059b
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFEnforceConstraint.java
@@ -0,0 +1,77 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.ql.udf.generic;
+
+import org.apache.hadoop.hive.ql.exec.Description;
+import org.apache.hadoop.hive.ql.exec.UDFArgumentException;
+import org.apache.hadoop.hive.ql.exec.UDFArgumentLengthException;
+import org.apache.hadoop.hive.ql.exec.errors.DataConstraintViolationError;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.BooleanObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
+import org.apache.hadoop.io.BooleanWritable;
+
+/**
+ * GenericUDFAbs.
+ *
+ */
+@Description(name = "enforce_constraint",
+    value = "_FUNC_(x) - Internal UDF to enforce CHECK and NOT NULL constraint",
+    extended = "For internal use only")
+public class GenericUDFEnforceConstraint extends GenericUDF {
+  private final BooleanWritable resultBool = new BooleanWritable();
+  private transient BooleanObjectInspector boi;
+
+  @Override
+  public ObjectInspector initialize(ObjectInspector[] arguments) throws UDFArgumentException {
+    if (arguments.length > 1) {
+      throw new UDFArgumentLengthException(
+          "Invalid number of arguments. enforce_constraint UDF expected one argument but received: "
+              + arguments.length);
+    }
+
+    boi = (BooleanObjectInspector) arguments[0];
+    return PrimitiveObjectInspectorFactory.writableBooleanObjectInspector;
+  }
+
+  @Override
+  public Object evaluate(DeferredObject[] arguments) throws HiveException {
+
+    Object a = arguments[0].get();
+    boolean result = boi.get(a);
+
+    if(!result) {
+      throw new DataConstraintViolationError(
+          "Either CHECK or NOT NULL constraint violated!");
+    }
+    resultBool.set(true);
+    return resultBool;
+  }
+
+  @Override
+  protected String getFuncName() {
+    return "enforce_constraint";
+  }
+
+  @Override
+  public String getDisplayString(String[] children) {
+    return getStandardDisplayString(getFuncName(), children);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/ba7155d3/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFEnforceNotNullConstraint.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFEnforceNotNullConstraint.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFEnforceNotNullConstraint.java
deleted file mode 100644
index 6c8c6fd..0000000
--- a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFEnforceNotNullConstraint.java
+++ /dev/null
@@ -1,77 +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.udf.generic;
-
-import org.apache.hadoop.hive.ql.exec.Description;
-import org.apache.hadoop.hive.ql.exec.UDFArgumentException;
-import org.apache.hadoop.hive.ql.exec.UDFArgumentLengthException;
-import org.apache.hadoop.hive.ql.exec.errors.DataConstraintViolationError;
-import org.apache.hadoop.hive.ql.metadata.HiveException;
-import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
-import org.apache.hadoop.hive.serde2.objectinspector.primitive.BooleanObjectInspector;
-import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
-import org.apache.hadoop.io.BooleanWritable;
-
-/**
- * GenericUDFAbs.
- *
- */
-@Description(name = "enforce_constraint",
-    value = "_FUNC_(x) - Internal UDF to enforce NOT NULL constraint",
-    extended = "For internal use only")
-public class GenericUDFEnforceNotNullConstraint extends GenericUDF {
-  private final BooleanWritable resultBool = new BooleanWritable();
-  private transient BooleanObjectInspector boi;
-
-  @Override
-  public ObjectInspector initialize(ObjectInspector[] arguments) throws UDFArgumentException {
-    if (arguments.length > 1) {
-      throw new UDFArgumentLengthException(
-          "Invalid number of arguments. enforce_constraint UDF expected one argument but received: "
-              + arguments.length);
-    }
-
-    boi = (BooleanObjectInspector) arguments[0];
-    return PrimitiveObjectInspectorFactory.writableBooleanObjectInspector;
-  }
-
-  @Override
-  public Object evaluate(DeferredObject[] arguments) throws HiveException {
-
-    Object a = arguments[0].get();
-    boolean result = boi.get(a);
-
-    if(!result) {
-      throw new DataConstraintViolationError(
-          "NOT NULL constraint violated!");
-    }
-    resultBool.set(true);
-    return resultBool;
-  }
-
-  @Override
-  protected String getFuncName() {
-    return "enforce_constraint";
-  }
-
-  @Override
-  public String getDisplayString(String[] children) {
-    return getStandardDisplayString(getFuncName(), children);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/ba7155d3/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFEnforceConstraint.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFEnforceConstraint.java b/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFEnforceConstraint.java
new file mode 100644
index 0000000..a0da723
--- /dev/null
+++ b/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFEnforceConstraint.java
@@ -0,0 +1,75 @@
+/*
+ * 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.udf.generic;
+
+import junit.framework.TestCase;
+import org.apache.hadoop.hive.ql.exec.errors.DataConstraintViolationError;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
+import org.apache.hadoop.io.BooleanWritable;
+
+/**
+ * Test class for {@link GenericUDFEnforceConstraint}.
+ */
+public class TestGenericUDFEnforceConstraint extends TestCase {
+
+  public void testNull() throws HiveException {
+    try {
+      GenericUDFEnforceConstraint udf = new GenericUDFEnforceConstraint();
+      ObjectInspector valueOI = PrimitiveObjectInspectorFactory.writableBooleanObjectInspector;
+      ObjectInspector[] arguments = {valueOI };
+      udf.initialize(arguments);
+
+      BooleanWritable input = new BooleanWritable(false);
+      GenericUDF.DeferredObject[] args = {new GenericUDF.DeferredJavaObject(input) };
+      udf.evaluate(args);
+      fail("Unreachable line");
+    } catch (DataConstraintViolationError e) {
+      //DataConstraintViolationError is expected
+      assertTrue(e.getMessage().contains("NOT NULL constraint violated!"));
+    }
+  }
+
+  public void testInvalidArgumentsLength() throws HiveException {
+    try {
+      GenericUDFEnforceConstraint udf = new GenericUDFEnforceConstraint();
+      ObjectInspector valueOI1 = PrimitiveObjectInspectorFactory.writableBooleanObjectInspector;
+      ObjectInspector valueOI2 = PrimitiveObjectInspectorFactory.writableBooleanObjectInspector;
+      ObjectInspector[] arguments = {valueOI1, valueOI2 };
+      udf.initialize(arguments);
+      fail("Unreachable line");
+    } catch (HiveException e) {
+      //HiveException is expected
+      assertTrue(e.getMessage().contains("Invalid number of arguments"));
+    }
+  }
+
+  public void testCorrect() throws HiveException {
+    GenericUDFEnforceConstraint udf = new GenericUDFEnforceConstraint();
+    ObjectInspector valueOI = PrimitiveObjectInspectorFactory.writableBooleanObjectInspector;
+    ObjectInspector[] arguments = {valueOI };
+    udf.initialize(arguments);
+
+    BooleanWritable input = new BooleanWritable(true);
+    GenericUDF.DeferredObject[] args = {new GenericUDF.DeferredJavaObject(input) };
+    BooleanWritable writable = (BooleanWritable) udf.evaluate(args);
+    assertTrue("Not expected result: expected [true] actual  [ " + writable.get() + " ]", writable.get());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/ba7155d3/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFEnforceNotNullConstraint.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFEnforceNotNullConstraint.java b/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFEnforceNotNullConstraint.java
deleted file mode 100644
index fc65bb6e..0000000
--- a/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFEnforceNotNullConstraint.java
+++ /dev/null
@@ -1,75 +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.udf.generic;
-
-import junit.framework.TestCase;
-import org.apache.hadoop.hive.ql.exec.errors.DataConstraintViolationError;
-import org.apache.hadoop.hive.ql.metadata.HiveException;
-import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
-import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
-import org.apache.hadoop.io.BooleanWritable;
-
-/**
- * Test class for {@link GenericUDFEnforceNotNullConstraint}.
- */
-public class TestGenericUDFEnforceNotNullConstraint extends TestCase {
-
-  public void testNull() throws HiveException {
-    try {
-      GenericUDFEnforceNotNullConstraint udf = new GenericUDFEnforceNotNullConstraint();
-      ObjectInspector valueOI = PrimitiveObjectInspectorFactory.writableBooleanObjectInspector;
-      ObjectInspector[] arguments = {valueOI };
-      udf.initialize(arguments);
-
-      BooleanWritable input = new BooleanWritable(false);
-      GenericUDF.DeferredObject[] args = {new GenericUDF.DeferredJavaObject(input) };
-      udf.evaluate(args);
-      fail("Unreachable line");
-    } catch (DataConstraintViolationError e) {
-      //DataConstraintViolationError is expected
-      assertTrue(e.getMessage().contains("NOT NULL constraint violated!"));
-    }
-  }
-
-  public void testInvalidArgumentsLength() throws HiveException {
-    try {
-      GenericUDFEnforceNotNullConstraint udf = new GenericUDFEnforceNotNullConstraint();
-      ObjectInspector valueOI1 = PrimitiveObjectInspectorFactory.writableBooleanObjectInspector;
-      ObjectInspector valueOI2 = PrimitiveObjectInspectorFactory.writableBooleanObjectInspector;
-      ObjectInspector[] arguments = {valueOI1, valueOI2 };
-      udf.initialize(arguments);
-      fail("Unreachable line");
-    } catch (HiveException e) {
-      //HiveException is expected
-      assertTrue(e.getMessage().contains("Invalid number of arguments"));
-    }
-  }
-
-  public void testCorrect() throws HiveException {
-    GenericUDFEnforceNotNullConstraint udf = new GenericUDFEnforceNotNullConstraint();
-    ObjectInspector valueOI = PrimitiveObjectInspectorFactory.writableBooleanObjectInspector;
-    ObjectInspector[] arguments = {valueOI };
-    udf.initialize(arguments);
-
-    BooleanWritable input = new BooleanWritable(true);
-    GenericUDF.DeferredObject[] args = {new GenericUDF.DeferredJavaObject(input) };
-    BooleanWritable writable = (BooleanWritable) udf.evaluate(args);
-    assertTrue("Not expected result: expected [true] actual  [ " + writable.get() + " ]", writable.get());
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/ba7155d3/ql/src/test/results/clientnegative/alter_notnull_constraint_violation.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/alter_notnull_constraint_violation.q.out b/ql/src/test/results/clientnegative/alter_notnull_constraint_violation.q.out
index 65195dc..2445b5d 100644
--- a/ql/src/test/results/clientnegative/alter_notnull_constraint_violation.q.out
+++ b/ql/src/test/results/clientnegative/alter_notnull_constraint_violation.q.out
@@ -24,4 +24,4 @@ POSTHOOK: query: alter table t1 change j j int constraint nn0 not null enforced
 POSTHOOK: type: ALTERTABLE_RENAMECOL
 POSTHOOK: Input: default@t1
 POSTHOOK: Output: default@t1
-FAILED: DataConstraintViolationError org.apache.hadoop.hive.ql.exec.errors.DataConstraintViolationError: NOT NULL constraint violated!
+FAILED: DataConstraintViolationError org.apache.hadoop.hive.ql.exec.errors.DataConstraintViolationError: Either CHECK or NOT NULL constraint violated!

http://git-wip-us.apache.org/repos/asf/hive/blob/ba7155d3/ql/src/test/results/clientnegative/insert_into_acid_notnull.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/insert_into_acid_notnull.q.out b/ql/src/test/results/clientnegative/insert_into_acid_notnull.q.out
index 172c933..777a087 100644
--- a/ql/src/test/results/clientnegative/insert_into_acid_notnull.q.out
+++ b/ql/src/test/results/clientnegative/insert_into_acid_notnull.q.out
@@ -10,4 +10,4 @@ POSTHOOK: query: create table acid_uami(i int,
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@acid_uami
-FAILED: DataConstraintViolationError org.apache.hadoop.hive.ql.exec.errors.DataConstraintViolationError: NOT NULL constraint violated!
+FAILED: DataConstraintViolationError org.apache.hadoop.hive.ql.exec.errors.DataConstraintViolationError: Either CHECK or NOT NULL constraint violated!

http://git-wip-us.apache.org/repos/asf/hive/blob/ba7155d3/ql/src/test/results/clientnegative/insert_into_notnull_constraint.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/insert_into_notnull_constraint.q.out b/ql/src/test/results/clientnegative/insert_into_notnull_constraint.q.out
index dd720fa..96feec0 100644
--- a/ql/src/test/results/clientnegative/insert_into_notnull_constraint.q.out
+++ b/ql/src/test/results/clientnegative/insert_into_notnull_constraint.q.out
@@ -6,4 +6,4 @@ POSTHOOK: query: create table nullConstraintCheck(i int NOT NULL enforced, j int
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@nullConstraintCheck
-FAILED: DataConstraintViolationError org.apache.hadoop.hive.ql.exec.errors.DataConstraintViolationError: NOT NULL constraint violated!
+FAILED: DataConstraintViolationError org.apache.hadoop.hive.ql.exec.errors.DataConstraintViolationError: Either CHECK or NOT NULL constraint violated!

http://git-wip-us.apache.org/repos/asf/hive/blob/ba7155d3/ql/src/test/results/clientnegative/insert_multi_into_notnull.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/insert_multi_into_notnull.q.out b/ql/src/test/results/clientnegative/insert_multi_into_notnull.q.out
index 1beeb26..74e112f 100644
--- a/ql/src/test/results/clientnegative/insert_multi_into_notnull.q.out
+++ b/ql/src/test/results/clientnegative/insert_multi_into_notnull.q.out
@@ -14,4 +14,4 @@ POSTHOOK: query: create table src_multi2 (i STRING, j STRING NOT NULL ENFORCED)
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@src_multi2
-FAILED: DataConstraintViolationError org.apache.hadoop.hive.ql.exec.errors.DataConstraintViolationError: NOT NULL constraint violated!
+FAILED: DataConstraintViolationError org.apache.hadoop.hive.ql.exec.errors.DataConstraintViolationError: Either CHECK or NOT NULL constraint violated!

http://git-wip-us.apache.org/repos/asf/hive/blob/ba7155d3/ql/src/test/results/clientnegative/insert_overwrite_notnull_constraint.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/insert_overwrite_notnull_constraint.q.out b/ql/src/test/results/clientnegative/insert_overwrite_notnull_constraint.q.out
index dd720fa..96feec0 100644
--- a/ql/src/test/results/clientnegative/insert_overwrite_notnull_constraint.q.out
+++ b/ql/src/test/results/clientnegative/insert_overwrite_notnull_constraint.q.out
@@ -6,4 +6,4 @@ POSTHOOK: query: create table nullConstraintCheck(i int NOT NULL enforced, j int
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@nullConstraintCheck
-FAILED: DataConstraintViolationError org.apache.hadoop.hive.ql.exec.errors.DataConstraintViolationError: NOT NULL constraint violated!
+FAILED: DataConstraintViolationError org.apache.hadoop.hive.ql.exec.errors.DataConstraintViolationError: Either CHECK or NOT NULL constraint violated!

http://git-wip-us.apache.org/repos/asf/hive/blob/ba7155d3/ql/src/test/results/clientnegative/update_notnull_constraint.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/update_notnull_constraint.q.out b/ql/src/test/results/clientnegative/update_notnull_constraint.q.out
index 8748681..86bfc67 100644
--- a/ql/src/test/results/clientnegative/update_notnull_constraint.q.out
+++ b/ql/src/test/results/clientnegative/update_notnull_constraint.q.out
@@ -21,4 +21,4 @@ POSTHOOK: Output: default@acid_uami
 POSTHOOK: Lineage: acid_uami.de SCRIPT []
 POSTHOOK: Lineage: acid_uami.i SCRIPT []
 POSTHOOK: Lineage: acid_uami.vc SCRIPT []
-FAILED: DataConstraintViolationError org.apache.hadoop.hive.ql.exec.errors.DataConstraintViolationError: NOT NULL constraint violated!
+FAILED: DataConstraintViolationError org.apache.hadoop.hive.ql.exec.errors.DataConstraintViolationError: Either CHECK or NOT NULL constraint violated!


[34/50] [abbrv] hive git commit: HIVE-19474 : Decimal type should be casted as part of the CTAS or INSERT Clause. (Slim Bouguerra via Jesus Camacho Rodriguez)

Posted by vg...@apache.org.
HIVE-19474 : Decimal type should be casted as part of the CTAS or INSERT Clause. (Slim Bouguerra via Jesus Camacho Rodriguez)

Signed-off-by: Ashutosh Chauhan <ha...@apache.org>


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

Branch: refs/heads/branch-3.0.0
Commit: 71d211d2dbf53031da27aec562b4fba48939841d
Parents: 1db0521
Author: Slim Bouguerra <sl...@gmail.com>
Authored: Mon May 14 09:34:14 2018 -0700
Committer: Vineet Garg <vg...@apache.org>
Committed: Mon May 14 11:13:14 2018 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hive/conf/HiveConf.java   |  3 -
 .../hive/druid/DruidStorageHandlerUtils.java    | 15 ++---
 .../hadoop/hive/druid/serde/DruidSerDe.java     | 60 ++++++++------------
 3 files changed, 28 insertions(+), 50 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/71d211d2/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
----------------------------------------------------------------------
diff --git a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
index 3bb1e80..e56c14f 100644
--- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
+++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
@@ -2682,9 +2682,6 @@ public class HiveConf extends Configuration {
             "Wait time in ms default to 30 seconds."
     ),
     HIVE_DRUID_BITMAP_FACTORY_TYPE("hive.druid.bitmap.type", "roaring", new PatternSet("roaring", "concise"), "Coding algorithm use to encode the bitmaps"),
-    HIVE_DRUID_APPROX_RESULT("hive.druid.approx.result", false,
-        "Whether to allow approximate results from druid. \n" +
-        "When set to true decimals will be stored as double and druid is allowed to return approximate results for decimal columns."),
     // For HBase storage handler
     HIVE_HBASE_WAL_ENABLED("hive.hbase.wal.enabled", true,
         "Whether writes to HBase should be forced to the write-ahead log. \n" +

http://git-wip-us.apache.org/repos/asf/hive/blob/71d211d2/druid-handler/src/java/org/apache/hadoop/hive/druid/DruidStorageHandlerUtils.java
----------------------------------------------------------------------
diff --git a/druid-handler/src/java/org/apache/hadoop/hive/druid/DruidStorageHandlerUtils.java b/druid-handler/src/java/org/apache/hadoop/hive/druid/DruidStorageHandlerUtils.java
index 93d3e5c..076f00a 100644
--- a/druid-handler/src/java/org/apache/hadoop/hive/druid/DruidStorageHandlerUtils.java
+++ b/druid-handler/src/java/org/apache/hadoop/hive/druid/DruidStorageHandlerUtils.java
@@ -817,8 +817,6 @@ public final class DruidStorageHandlerUtils {
     // Default, all columns that are not metrics or timestamp, are treated as dimensions
     final List<DimensionSchema> dimensions = new ArrayList<>();
     ImmutableList.Builder<AggregatorFactory> aggregatorFactoryBuilder = ImmutableList.builder();
-    final boolean approximationAllowed = HiveConf
-        .getBoolVar(jc, HiveConf.ConfVars.HIVE_DRUID_APPROX_RESULT);
     for (int i = 0; i < columnTypes.size(); i++) {
       final PrimitiveObjectInspector.PrimitiveCategory primitiveCategory = ((PrimitiveTypeInfo) columnTypes
           .get(i)).getPrimitiveCategory();
@@ -835,15 +833,10 @@ public final class DruidStorageHandlerUtils {
         af = new DoubleSumAggregatorFactory(columnNames.get(i), columnNames.get(i));
         break;
       case DECIMAL:
-        if (approximationAllowed) {
-          af = new DoubleSumAggregatorFactory(columnNames.get(i), columnNames.get(i));
-        } else {
-          throw new UnsupportedOperationException(
-              String.format("Druid does not support decimal column type." +
-                      "Either cast column [%s] to double or Enable Approximate Result for Druid by setting property [%s] to true",
-                  columnNames.get(i), HiveConf.ConfVars.HIVE_DRUID_APPROX_RESULT.varname));
-        }
-        break;
+        throw new UnsupportedOperationException(
+            String.format("Druid does not support decimal column type cast column "
+                + "[%s] to double", columnNames.get(i)));
+
       case TIMESTAMP:
         // Granularity column
         String tColumnName = columnNames.get(i);

http://git-wip-us.apache.org/repos/asf/hive/blob/71d211d2/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidSerDe.java
----------------------------------------------------------------------
diff --git a/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidSerDe.java b/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidSerDe.java
index d991adb..5f76579 100644
--- a/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidSerDe.java
+++ b/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidSerDe.java
@@ -17,26 +17,17 @@
  */
 package org.apache.hadoop.hive.druid.serde;
 
-import java.io.IOException;
-import java.io.InputStream;
-import java.sql.Timestamp;
-import java.time.Instant;
-import java.time.ZonedDateTime;
-import java.time.format.DateTimeFormatter;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Properties;
-import java.util.stream.Collectors;
-
+import com.fasterxml.jackson.core.type.TypeReference;
+import com.google.common.base.Function;
 import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import io.druid.query.Druids;
+import io.druid.query.Druids.SegmentMetadataQueryBuilder;
+import io.druid.query.metadata.metadata.ColumnAnalysis;
+import io.druid.query.metadata.metadata.SegmentAnalysis;
+import io.druid.query.metadata.metadata.SegmentMetadataQuery;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.common.type.HiveChar;
-import org.apache.hadoop.hive.common.type.HiveDecimal;
 import org.apache.hadoop.hive.common.type.HiveVarchar;
 import org.apache.hadoop.hive.common.type.TimestampTZ;
 import org.apache.hadoop.hive.conf.Constants;
@@ -53,7 +44,6 @@ import org.apache.hadoop.hive.serde2.SerDeStats;
 import org.apache.hadoop.hive.serde2.io.ByteWritable;
 import org.apache.hadoop.hive.serde2.io.DoubleWritable;
 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.io.ShortWritable;
 import org.apache.hadoop.hive.serde2.io.TimestampLocalTZWritable;
@@ -67,7 +57,6 @@ import org.apache.hadoop.hive.serde2.objectinspector.primitive.ByteObjectInspect
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.DoubleObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.FloatObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.HiveCharObjectInspector;
-import org.apache.hadoop.hive.serde2.objectinspector.primitive.HiveDecimalObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.HiveVarcharObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.IntObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.LongObjectInspector;
@@ -92,15 +81,21 @@ import org.apache.hadoop.util.StringUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.fasterxml.jackson.core.type.TypeReference;
-import com.google.common.base.Function;
-import com.google.common.collect.Lists;
-
-import io.druid.query.Druids;
-import io.druid.query.Druids.SegmentMetadataQueryBuilder;
-import io.druid.query.metadata.metadata.ColumnAnalysis;
-import io.druid.query.metadata.metadata.SegmentAnalysis;
-import io.druid.query.metadata.metadata.SegmentMetadataQuery;
+import java.io.IOException;
+import java.io.InputStream;
+import java.sql.Timestamp;
+import java.time.Instant;
+import java.time.ZonedDateTime;
+import java.time.format.DateTimeFormatter;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Properties;
+import java.util.stream.Collectors;
 
 /**
  * DruidSerDe that is used to  deserialize objects from a Druid data source.
@@ -350,10 +345,6 @@ public class DruidSerDe extends AbstractSerDe {
           res = ((DoubleObjectInspector) fields.get(i).getFieldObjectInspector())
                   .get(values.get(i));
           break;
-        case DECIMAL:
-          res = ((HiveDecimalObjectInspector) fields.get(i).getFieldObjectInspector())
-                  .getPrimitiveJavaObject(values.get(i)).doubleValue();
-          break;
         case CHAR:
           res = ((HiveCharObjectInspector) fields.get(i).getFieldObjectInspector())
                   .getPrimitiveJavaObject(values.get(i)).getValue();
@@ -371,7 +362,7 @@ public class DruidSerDe extends AbstractSerDe {
                   .get(values.get(i));
           break;
         default:
-          throw new SerDeException("Unknown type: " + types[i].getPrimitiveCategory());
+          throw new SerDeException("Unsupported type: " + types[i].getPrimitiveCategory());
       }
       value.put(columns[i], res);
     }
@@ -452,9 +443,6 @@ public class DruidSerDe extends AbstractSerDe {
         case DOUBLE:
           output.add(new DoubleWritable(((Number) value).doubleValue()));
           break;
-        case DECIMAL:
-          output.add(new HiveDecimalWritable(HiveDecimal.create(((Number) value).doubleValue())));
-          break;
         case CHAR:
           output.add(
               new HiveCharWritable(


[33/50] [abbrv] hive git commit: HIVE-19159: TestMTQueries.testMTQueries1 failure (Laszlo Bodor via Zoltan Haindrich)

Posted by vg...@apache.org.
HIVE-19159: TestMTQueries.testMTQueries1 failure (Laszlo Bodor via Zoltan Haindrich)

Signed-off-by: Zoltan Haindrich <ki...@rxd.hu>


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

Branch: refs/heads/branch-3.0.0
Commit: 1db0521b593fd1e354c75fd719a98d231ded84d6
Parents: 685fc9f
Author: Laszlo Bodor <bo...@gmail.com>
Authored: Mon May 14 15:46:50 2018 +0200
Committer: Zoltan Haindrich <ki...@rxd.hu>
Committed: Mon May 14 15:46:50 2018 +0200

----------------------------------------------------------------------
 .../apache/hadoop/hive/ql/TestMTQueries.java    |  2 +-
 .../org/apache/hadoop/hive/ql/QTestUtil.java    | 93 +++++++++++---------
 2 files changed, 50 insertions(+), 45 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/1db0521b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/TestMTQueries.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/TestMTQueries.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/TestMTQueries.java
index 4838856..6ed872d 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/TestMTQueries.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/TestMTQueries.java
@@ -42,7 +42,7 @@ public class TestMTQueries extends BaseTestQueries {
       // derby fails creating multiple stats aggregator concurrently
       util.getConf().setBoolean("hive.exec.submitviachild", true);
       util.getConf().setBoolean("hive.exec.submit.local.task.via.child", true);
-      util.getConf().setBoolean("hive.vectorized.execution.enabled", false);
+      util.getConf().setBoolean("hive.vectorized.execution.enabled", true);
       util.getConf().set("hive.stats.dbclass", "fs");
       util.getConf().set("hive.mapred.mode", "nonstrict");
       util.getConf().set("hive.stats.column.autogather", "false");

http://git-wip-us.apache.org/repos/asf/hive/blob/1db0521b/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
----------------------------------------------------------------------
diff --git a/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java b/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
index 58680fe..f6729a9 100644
--- a/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
+++ b/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
@@ -36,6 +36,7 @@ import java.io.OutputStream;
 import java.io.PrintStream;
 import java.io.Serializable;
 import java.io.StringWriter;
+import java.io.UnsupportedEncodingException;
 import java.net.URL;
 import java.nio.charset.StandardCharsets;
 import java.sql.SQLException;
@@ -1200,11 +1201,13 @@ public class QTestUtil {
 
     DatasetCollection datasets = parser.getDatasets();
     for (String table : datasets.getTables()){
-      initDataset(table);
+      synchronized (QTestUtil.class){
+        initDataset(table);
+      }
     }
   }
 
-  private void initDataset(String table) {
+  protected void initDataset(String table) {
     if (getSrcTables().contains(table)){
       return;
     }
@@ -1270,7 +1273,7 @@ public class QTestUtil {
     initDataSetForTest(file);
 
     HiveConf.setVar(conf, HiveConf.ConfVars.HIVE_AUTHENTICATOR_MANAGER,
-    "org.apache.hadoop.hive.ql.security.DummyAuthenticator");
+        "org.apache.hadoop.hive.ql.security.DummyAuthenticator");
     Utilities.clearWorkMap(conf);
     CliSessionState ss = new CliSessionState(conf);
     assert ss != null;
@@ -1287,6 +1290,30 @@ public class QTestUtil {
     }
 
     File outf = new File(logDir, stdoutName);
+
+    setSessionOutputs(fileName, ss, outf);
+
+    SessionState oldSs = SessionState.get();
+
+    boolean canReuseSession = !qNoSessionReuseQuerySet.contains(fileName);
+    restartSessions(canReuseSession, ss, oldSs);
+
+    closeSession(oldSs);
+
+    SessionState.start(ss);
+
+    cliDriver = new CliDriver();
+
+    if (fileName.equals("init_file.q")) {
+      ss.initFiles.add(AbstractCliConfig.HIVE_ROOT + "/data/scripts/test_init_file.sql");
+    }
+    cliDriver.processInitFiles(ss);
+
+    return outf.getAbsolutePath();
+  }
+
+  private void setSessionOutputs(String fileName, CliSessionState ss, File outf)
+      throws FileNotFoundException, Exception, UnsupportedEncodingException {
     OutputStream fo = new BufferedOutputStream(new FileOutputStream(outf));
     if (qSortQuerySet.contains(fileName)) {
       ss.out = new SortPrintStream(fo, "UTF-8");
@@ -1299,10 +1326,12 @@ public class QTestUtil {
     }
     ss.err = new CachingPrintStream(fo, true, "UTF-8");
     ss.setIsSilent(true);
-    SessionState oldSs = SessionState.get();
+  }
 
-    boolean canReuseSession = !qNoSessionReuseQuerySet.contains(fileName);
-    if (oldSs != null && canReuseSession && clusterType.getCoreClusterType() == CoreClusterType.TEZ) {
+  private void restartSessions(boolean canReuseSession, CliSessionState ss, SessionState oldSs)
+      throws IOException {
+    if (oldSs != null && canReuseSession
+        && clusterType.getCoreClusterType() == CoreClusterType.TEZ) {
       // Copy the tezSessionState from the old CliSessionState.
       TezSessionState tezSessionState = oldSs.getTezSession();
       oldSs.setTezSession(null);
@@ -1316,27 +1345,9 @@ public class QTestUtil {
       oldSs.setSparkSession(null);
       oldSs.close();
     }
-
-    if (oldSs != null && oldSs.out != null && oldSs.out != System.out) {
-      oldSs.out.close();
-    }
-    if (oldSs != null) {
-      oldSs.close();
-    }
-    SessionState.start(ss);
-
-    cliDriver = new CliDriver();
-
-    if (fileName.equals("init_file.q")) {
-      ss.initFiles.add(AbstractCliConfig.HIVE_ROOT + "/data/scripts/test_init_file.sql");
-    }
-    cliDriver.processInitFiles(ss);
-
-    return outf.getAbsolutePath();
   }
 
-  private CliSessionState startSessionState(boolean canReuseSession)
-      throws IOException {
+  private CliSessionState startSessionState(boolean canReuseSession) throws IOException {
 
     HiveConf.setVar(conf, HiveConf.ConfVars.HIVE_AUTHENTICATOR_MANAGER,
         "org.apache.hadoop.hive.ql.security.DummyAuthenticator");
@@ -1350,26 +1361,10 @@ public class QTestUtil {
     ss.err = System.out;
 
     SessionState oldSs = SessionState.get();
-    if (oldSs != null && canReuseSession && clusterType.getCoreClusterType() == CoreClusterType.TEZ) {
-      // Copy the tezSessionState from the old CliSessionState.
-      TezSessionState tezSessionState = oldSs.getTezSession();
-      ss.setTezSession(tezSessionState);
-      oldSs.setTezSession(null);
-      oldSs.close();
-    }
 
-    if (oldSs != null && clusterType.getCoreClusterType() == CoreClusterType.SPARK) {
-      sparkSession = oldSs.getSparkSession();
-      ss.setSparkSession(sparkSession);
-      oldSs.setSparkSession(null);
-      oldSs.close();
-    }
-    if (oldSs != null && oldSs.out != null && oldSs.out != System.out) {
-      oldSs.out.close();
-    }
-    if (oldSs != null) {
-      oldSs.close();
-    }
+    restartSessions(canReuseSession, ss, oldSs);
+
+    closeSession(oldSs);
     SessionState.start(ss);
 
     isSessionStateStarted = true;
@@ -1378,6 +1373,15 @@ public class QTestUtil {
     return ss;
   }
 
+  private void closeSession(SessionState oldSs) throws IOException {
+    if (oldSs != null && oldSs.out != null && oldSs.out != System.out) {
+      oldSs.out.close();
+    }
+    if (oldSs != null) {
+      oldSs.close();
+    }
+  }
+
   public int executeAdhocCommand(String q) {
     if (!q.contains(";")) {
       return -1;
@@ -1986,6 +1990,7 @@ public class QTestUtil {
     @Override
     public void run() {
       try {
+        qt.startSessionState(false);
         // assumption is that environment has already been cleaned once globally
         // hence each thread does not call cleanUp() and createSources() again
         qt.cliInit(file, false);


[04/50] [abbrv] hive git commit: HIVE-19357: Vectorization: assert_true HiveException erroneously gets suppressed to NULL (Matt McCline, reviewed by Zoltan Haindrich)

Posted by vg...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/gen/vectorization/ExpressionTemplates/IntervalYearMonthScalarArithmeticTimestampColumn.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/IntervalYearMonthScalarArithmeticTimestampColumn.txt b/ql/src/gen/vectorization/ExpressionTemplates/IntervalYearMonthScalarArithmeticTimestampColumn.txt
index 157e95e..9f18c5a 100644
--- a/ql/src/gen/vectorization/ExpressionTemplates/IntervalYearMonthScalarArithmeticTimestampColumn.txt
+++ b/ql/src/gen/vectorization/ExpressionTemplates/IntervalYearMonthScalarArithmeticTimestampColumn.txt
@@ -36,6 +36,7 @@ import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.NullUtil;
 import org.apache.hadoop.hive.ql.util.DateTimeMath;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 /**
  * Generated from template TimestampScalarArithmeticIntervalYearMonthColumn.txt.
@@ -71,7 +72,7 @@ public class <ClassName> extends VectorExpression {
    *
    * @batch a package of rows with each column stored in a vector
    */
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     // return immediately if batch is empty
     final int n = batch.size;

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/gen/vectorization/ExpressionTemplates/LongDoubleColumnCompareTimestampColumn.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/LongDoubleColumnCompareTimestampColumn.txt b/ql/src/gen/vectorization/ExpressionTemplates/LongDoubleColumnCompareTimestampColumn.txt
index e3f36b9..703efdd 100644
--- a/ql/src/gen/vectorization/ExpressionTemplates/LongDoubleColumnCompareTimestampColumn.txt
+++ b/ql/src/gen/vectorization/ExpressionTemplates/LongDoubleColumnCompareTimestampColumn.txt
@@ -25,6 +25,7 @@ import org.apache.hadoop.hive.ql.exec.vector.expressions.NullUtil;
 import org.apache.hadoop.hive.ql.exec.vector.*;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 /**
  * Generated from template ColumnCompareTimestampColumn.txt, which covers binary arithmetic
@@ -52,7 +53,7 @@ public class <ClassName> extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     // return immediately if batch is empty
     final int n = batch.size;

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/gen/vectorization/ExpressionTemplates/LongDoubleColumnCompareTimestampScalar.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/LongDoubleColumnCompareTimestampScalar.txt b/ql/src/gen/vectorization/ExpressionTemplates/LongDoubleColumnCompareTimestampScalar.txt
index 2957c73..19263d9 100644
--- a/ql/src/gen/vectorization/ExpressionTemplates/LongDoubleColumnCompareTimestampScalar.txt
+++ b/ql/src/gen/vectorization/ExpressionTemplates/LongDoubleColumnCompareTimestampScalar.txt
@@ -25,6 +25,7 @@ import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
 import org.apache.hadoop.hive.ql.exec.vector.*;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 /**
  * Generated from template ColumnCompareTimestampScalar.txt, which covers binary comparison
@@ -53,7 +54,7 @@ public class <ClassName> extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     // return immediately if batch is empty
     final int n = batch.size;

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/gen/vectorization/ExpressionTemplates/LongDoubleScalarCompareTimestampColumn.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/LongDoubleScalarCompareTimestampColumn.txt b/ql/src/gen/vectorization/ExpressionTemplates/LongDoubleScalarCompareTimestampColumn.txt
index bb7f57d..067d21f 100644
--- a/ql/src/gen/vectorization/ExpressionTemplates/LongDoubleScalarCompareTimestampColumn.txt
+++ b/ql/src/gen/vectorization/ExpressionTemplates/LongDoubleScalarCompareTimestampColumn.txt
@@ -26,6 +26,7 @@ import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.*;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 /**
  * Generated from template ScalarCompareTimestamp.txt, which covers comparison
@@ -54,7 +55,7 @@ public class <ClassName> extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     // return immediately if batch is empty
     final int n = batch.size;

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/gen/vectorization/ExpressionTemplates/ScalarArithmeticColumn.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/ScalarArithmeticColumn.txt b/ql/src/gen/vectorization/ExpressionTemplates/ScalarArithmeticColumn.txt
index 695a063..be8001d 100644
--- a/ql/src/gen/vectorization/ExpressionTemplates/ScalarArithmeticColumn.txt
+++ b/ql/src/gen/vectorization/ExpressionTemplates/ScalarArithmeticColumn.txt
@@ -33,6 +33,7 @@ import org.apache.hadoop.hive.ql.exec.vector.DoubleColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.NullUtil;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 /**
  * Generated from template ScalarArithmeticColumn.txt.
@@ -66,7 +67,7 @@ public class <ClassName> extends VectorExpression {
    *
    * @batch a package of rows with each column stored in a vector
    */
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     // return immediately if batch is empty
     final int n = batch.size;

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/gen/vectorization/ExpressionTemplates/ScalarArithmeticColumnDecimal.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/ScalarArithmeticColumnDecimal.txt b/ql/src/gen/vectorization/ExpressionTemplates/ScalarArithmeticColumnDecimal.txt
index a0dbea6..7877127 100644
--- a/ql/src/gen/vectorization/ExpressionTemplates/ScalarArithmeticColumnDecimal.txt
+++ b/ql/src/gen/vectorization/ExpressionTemplates/ScalarArithmeticColumnDecimal.txt
@@ -28,6 +28,7 @@ import org.apache.hadoop.hive.ql.exec.vector.expressions.DecimalUtil;
 import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
 import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
 import org.apache.hadoop.hive.common.type.HiveDecimal;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 /**
  * Generated from template ScalarArithmeticColumnDecimal.txt, which covers binary arithmetic 
@@ -55,7 +56,7 @@ public class <ClassName> extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     // return immediately if batch is empty
     final int n = batch.size;

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/gen/vectorization/ExpressionTemplates/ScalarCompareColumn.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/ScalarCompareColumn.txt b/ql/src/gen/vectorization/ExpressionTemplates/ScalarCompareColumn.txt
index 06cd7e6..753f061 100644
--- a/ql/src/gen/vectorization/ExpressionTemplates/ScalarCompareColumn.txt
+++ b/ql/src/gen/vectorization/ExpressionTemplates/ScalarCompareColumn.txt
@@ -26,6 +26,7 @@ import org.apache.hadoop.hive.ql.exec.vector.<InputColumnVectorType>;
 import org.apache.hadoop.hive.ql.exec.vector.<OutputColumnVectorType>;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 /**
  * Generated from template ColumnCompareScalar.txt, which covers binary comparison 
@@ -54,7 +55,7 @@ public class <ClassName> extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     // return immediately if batch is empty
     final int n = batch.size;

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/gen/vectorization/ExpressionTemplates/ScalarDivideColumn.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/ScalarDivideColumn.txt b/ql/src/gen/vectorization/ExpressionTemplates/ScalarDivideColumn.txt
index a5de652..95e4ce1 100644
--- a/ql/src/gen/vectorization/ExpressionTemplates/ScalarDivideColumn.txt
+++ b/ql/src/gen/vectorization/ExpressionTemplates/ScalarDivideColumn.txt
@@ -32,6 +32,7 @@ import org.apache.hadoop.hive.ql.exec.vector.DoubleColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.NullUtil;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 /**
  * Generated from template ScalarArithmeticColumn.txt.
@@ -65,7 +66,7 @@ public class <ClassName> extends VectorExpression {
    *
    * @batch a package of rows with each column stored in a vector
    */
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     // return immediately if batch is empty
     final int n = batch.size;

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/gen/vectorization/ExpressionTemplates/ScalarDivideColumnDecimal.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/ScalarDivideColumnDecimal.txt b/ql/src/gen/vectorization/ExpressionTemplates/ScalarDivideColumnDecimal.txt
index b4ec35c..b3f1031 100644
--- a/ql/src/gen/vectorization/ExpressionTemplates/ScalarDivideColumnDecimal.txt
+++ b/ql/src/gen/vectorization/ExpressionTemplates/ScalarDivideColumnDecimal.txt
@@ -28,6 +28,7 @@ import org.apache.hadoop.hive.ql.exec.vector.expressions.DecimalUtil;
 import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
 import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
 import org.apache.hadoop.hive.common.type.HiveDecimal;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 /**
  * Generated from template ScalarDivideColumnDecimal.txt, which covers binary arithmetic
@@ -55,7 +56,7 @@ public class <ClassName> extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     // return immediately if batch is empty
     final int n = batch.size;

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/gen/vectorization/ExpressionTemplates/StringGroupColumnCompareStringGroupColumn.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/StringGroupColumnCompareStringGroupColumn.txt b/ql/src/gen/vectorization/ExpressionTemplates/StringGroupColumnCompareStringGroupColumn.txt
index 573af7f..8a0c8c1 100644
--- a/ql/src/gen/vectorization/ExpressionTemplates/StringGroupColumnCompareStringGroupColumn.txt
+++ b/ql/src/gen/vectorization/ExpressionTemplates/StringGroupColumnCompareStringGroupColumn.txt
@@ -26,6 +26,7 @@ import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 /**
  * Filter the rows in a batch by comparing one string column to another. 
@@ -53,7 +54,7 @@ public class <ClassName> extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     // return immediately if batch is empty
     final int n = batch.size;

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/gen/vectorization/ExpressionTemplates/StringGroupColumnCompareStringGroupScalarBase.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/StringGroupColumnCompareStringGroupScalarBase.txt b/ql/src/gen/vectorization/ExpressionTemplates/StringGroupColumnCompareStringGroupScalarBase.txt
index cb6cf4e..48913a3 100644
--- a/ql/src/gen/vectorization/ExpressionTemplates/StringGroupColumnCompareStringGroupScalarBase.txt
+++ b/ql/src/gen/vectorization/ExpressionTemplates/StringGroupColumnCompareStringGroupScalarBase.txt
@@ -27,6 +27,7 @@ import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 /**
  * This is a generated class to evaluate a <OperatorSymbol> comparison on a vector of strings.
@@ -53,7 +54,7 @@ public abstract class <ClassName> extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     // return immediately if batch is empty
     final int n = batch.size;

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/gen/vectorization/ExpressionTemplates/StringGroupScalarCompareStringGroupColumnBase.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/StringGroupScalarCompareStringGroupColumnBase.txt b/ql/src/gen/vectorization/ExpressionTemplates/StringGroupScalarCompareStringGroupColumnBase.txt
index 74d8b48..3cf4a2e 100644
--- a/ql/src/gen/vectorization/ExpressionTemplates/StringGroupScalarCompareStringGroupColumnBase.txt
+++ b/ql/src/gen/vectorization/ExpressionTemplates/StringGroupScalarCompareStringGroupColumnBase.txt
@@ -27,6 +27,7 @@ import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 /**
  * This is a generated class to evaluate a <OperatorSymbol> comparison on a vector of strings.
@@ -54,7 +55,7 @@ public abstract class <ClassName> extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     // return immediately if batch is empty
     final int n = batch.size;

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/gen/vectorization/ExpressionTemplates/TimestampColumnArithmeticDateColumn.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/TimestampColumnArithmeticDateColumn.txt b/ql/src/gen/vectorization/ExpressionTemplates/TimestampColumnArithmeticDateColumn.txt
index 27d8a3d..45f6408 100644
--- a/ql/src/gen/vectorization/ExpressionTemplates/TimestampColumnArithmeticDateColumn.txt
+++ b/ql/src/gen/vectorization/ExpressionTemplates/TimestampColumnArithmeticDateColumn.txt
@@ -28,6 +28,7 @@ import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
 import org.apache.hadoop.hive.ql.util.DateTimeMath;
 import org.apache.hadoop.hive.serde2.io.DateWritable;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 /**
  * Generated from template TimestampColumnArithmeticDateColumn.txt, which covers binary arithmetic
@@ -58,7 +59,7 @@ public class <ClassName> extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     // return immediately if batch is empty
     final int n = batch.size;

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/gen/vectorization/ExpressionTemplates/TimestampColumnArithmeticDateScalar.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/TimestampColumnArithmeticDateScalar.txt b/ql/src/gen/vectorization/ExpressionTemplates/TimestampColumnArithmeticDateScalar.txt
index 799daf2..0bbdce7 100644
--- a/ql/src/gen/vectorization/ExpressionTemplates/TimestampColumnArithmeticDateScalar.txt
+++ b/ql/src/gen/vectorization/ExpressionTemplates/TimestampColumnArithmeticDateScalar.txt
@@ -30,6 +30,7 @@ import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
 import org.apache.hadoop.hive.ql.exec.vector.*;
 import org.apache.hadoop.hive.ql.util.DateTimeMath;
 import org.apache.hadoop.hive.serde2.io.DateWritable;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 /**
  * Generated from template TimestampColumnArithmeticDateScalar.txt, which covers binary arithmetic
@@ -60,7 +61,7 @@ public class <ClassName> extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     // return immediately if batch is empty
     final int n = batch.size;

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/gen/vectorization/ExpressionTemplates/TimestampColumnArithmeticIntervalYearMonthColumn.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/TimestampColumnArithmeticIntervalYearMonthColumn.txt b/ql/src/gen/vectorization/ExpressionTemplates/TimestampColumnArithmeticIntervalYearMonthColumn.txt
index f894bcf..982a880 100644
--- a/ql/src/gen/vectorization/ExpressionTemplates/TimestampColumnArithmeticIntervalYearMonthColumn.txt
+++ b/ql/src/gen/vectorization/ExpressionTemplates/TimestampColumnArithmeticIntervalYearMonthColumn.txt
@@ -27,6 +27,7 @@ import org.apache.hadoop.hive.ql.exec.vector.*;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
 import org.apache.hadoop.hive.ql.util.DateTimeMath;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 /**
  * Generated from template TimestampColumnArithmeticIntervalYearMonthColumn.txt, which covers binary arithmetic
@@ -57,7 +58,7 @@ public class <ClassName> extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     // return immediately if batch is empty
     final int n = batch.size;

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/gen/vectorization/ExpressionTemplates/TimestampColumnArithmeticIntervalYearMonthScalar.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/TimestampColumnArithmeticIntervalYearMonthScalar.txt b/ql/src/gen/vectorization/ExpressionTemplates/TimestampColumnArithmeticIntervalYearMonthScalar.txt
index 0e2cd13..8edd9eb 100644
--- a/ql/src/gen/vectorization/ExpressionTemplates/TimestampColumnArithmeticIntervalYearMonthScalar.txt
+++ b/ql/src/gen/vectorization/ExpressionTemplates/TimestampColumnArithmeticIntervalYearMonthScalar.txt
@@ -28,6 +28,7 @@ import org.apache.hadoop.hive.ql.exec.vector.expressions.NullUtil;
 import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
 import org.apache.hadoop.hive.ql.exec.vector.*;
 import org.apache.hadoop.hive.ql.util.DateTimeMath;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 /**
  * Generated from template TimestampColumnArithmeticIntervalYearMonthScalar.txt, which covers binary arithmetic
@@ -57,7 +58,7 @@ public class <ClassName> extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     // return immediately if batch is empty
     final int n = batch.size;

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/gen/vectorization/ExpressionTemplates/TimestampColumnArithmeticTimestampColumn.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/TimestampColumnArithmeticTimestampColumn.txt b/ql/src/gen/vectorization/ExpressionTemplates/TimestampColumnArithmeticTimestampColumn.txt
index 4240994..2afc778 100644
--- a/ql/src/gen/vectorization/ExpressionTemplates/TimestampColumnArithmeticTimestampColumn.txt
+++ b/ql/src/gen/vectorization/ExpressionTemplates/TimestampColumnArithmeticTimestampColumn.txt
@@ -27,6 +27,7 @@ import org.apache.hadoop.hive.ql.exec.vector.*;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
 import org.apache.hadoop.hive.ql.util.DateTimeMath;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 /**
  * Generated from template TimestampColumnArithmeticTimestampColumnBase.txt, which covers binary arithmetic
@@ -56,7 +57,7 @@ public class <ClassName> extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     // return immediately if batch is empty
     final int n = batch.size;

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/gen/vectorization/ExpressionTemplates/TimestampColumnArithmeticTimestampScalar.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/TimestampColumnArithmeticTimestampScalar.txt b/ql/src/gen/vectorization/ExpressionTemplates/TimestampColumnArithmeticTimestampScalar.txt
index bcb8fd1..6532fcf 100644
--- a/ql/src/gen/vectorization/ExpressionTemplates/TimestampColumnArithmeticTimestampScalar.txt
+++ b/ql/src/gen/vectorization/ExpressionTemplates/TimestampColumnArithmeticTimestampScalar.txt
@@ -29,6 +29,7 @@ import org.apache.hadoop.hive.ql.exec.vector.expressions.NullUtil;
 import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
 import org.apache.hadoop.hive.ql.exec.vector.*;
 import org.apache.hadoop.hive.ql.util.DateTimeMath;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 /**
  * Generated from template TimestampColumnArithmeticTimestampScalar.txt, which covers binary arithmetic
@@ -58,7 +59,7 @@ public class <ClassName> extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     // return immediately if batch is empty
     final int n = batch.size;

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/gen/vectorization/ExpressionTemplates/TimestampColumnCompareLongDoubleColumn.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/TimestampColumnCompareLongDoubleColumn.txt b/ql/src/gen/vectorization/ExpressionTemplates/TimestampColumnCompareLongDoubleColumn.txt
index 7e65b9b..d17437b 100644
--- a/ql/src/gen/vectorization/ExpressionTemplates/TimestampColumnCompareLongDoubleColumn.txt
+++ b/ql/src/gen/vectorization/ExpressionTemplates/TimestampColumnCompareLongDoubleColumn.txt
@@ -23,6 +23,7 @@ import org.apache.hadoop.hive.ql.exec.vector.expressions.NullUtil;
 import org.apache.hadoop.hive.ql.exec.vector.*;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 /**
  * Generated from template ColumnArithmeticColumn.txt, which covers binary arithmetic
@@ -50,7 +51,7 @@ public class <ClassName> extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     // return immediately if batch is empty
     final int n = batch.size;

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/gen/vectorization/ExpressionTemplates/TimestampColumnCompareLongDoubleScalar.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/TimestampColumnCompareLongDoubleScalar.txt b/ql/src/gen/vectorization/ExpressionTemplates/TimestampColumnCompareLongDoubleScalar.txt
index b1e92e0..e2f6c7b 100644
--- a/ql/src/gen/vectorization/ExpressionTemplates/TimestampColumnCompareLongDoubleScalar.txt
+++ b/ql/src/gen/vectorization/ExpressionTemplates/TimestampColumnCompareLongDoubleScalar.txt
@@ -27,6 +27,7 @@ import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.*;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 /**
  * Generated from template TimestampColumnCompareScalar.txt, which covers comparison
@@ -55,7 +56,7 @@ public class <ClassName> extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     // return immediately if batch is empty
     final int n = batch.size;

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/gen/vectorization/ExpressionTemplates/TimestampColumnCompareTimestampColumn.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/TimestampColumnCompareTimestampColumn.txt b/ql/src/gen/vectorization/ExpressionTemplates/TimestampColumnCompareTimestampColumn.txt
index b81b805..a9a8659 100644
--- a/ql/src/gen/vectorization/ExpressionTemplates/TimestampColumnCompareTimestampColumn.txt
+++ b/ql/src/gen/vectorization/ExpressionTemplates/TimestampColumnCompareTimestampColumn.txt
@@ -26,6 +26,7 @@ import org.apache.hadoop.hive.ql.exec.vector.expressions.NullUtil;
 import org.apache.hadoop.hive.ql.exec.vector.*;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 /**
  * Generated from template TimestampColumnCompareColumn.txt, which covers comparision
@@ -53,7 +54,7 @@ public class <ClassName> extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     // return immediately if batch is empty
     final int n = batch.size;

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/gen/vectorization/ExpressionTemplates/TimestampColumnCompareTimestampScalar.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/TimestampColumnCompareTimestampScalar.txt b/ql/src/gen/vectorization/ExpressionTemplates/TimestampColumnCompareTimestampScalar.txt
index cee680a..dce87f4 100644
--- a/ql/src/gen/vectorization/ExpressionTemplates/TimestampColumnCompareTimestampScalar.txt
+++ b/ql/src/gen/vectorization/ExpressionTemplates/TimestampColumnCompareTimestampScalar.txt
@@ -27,6 +27,7 @@ import org.apache.hadoop.hive.ql.exec.vector.expressions.NullUtil;
 import org.apache.hadoop.hive.ql.exec.vector.*;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 /**
  * Generated from template TimestampColumnCompareTimestampScalar.txt, which covers binary comparison
@@ -55,7 +56,7 @@ public class <ClassName> extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     // return immediately if batch is empty
     final int n = batch.size;

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/gen/vectorization/ExpressionTemplates/TimestampScalarArithmeticDateColumn.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/TimestampScalarArithmeticDateColumn.txt b/ql/src/gen/vectorization/ExpressionTemplates/TimestampScalarArithmeticDateColumn.txt
index b50cbc8..9a21cda 100644
--- a/ql/src/gen/vectorization/ExpressionTemplates/TimestampScalarArithmeticDateColumn.txt
+++ b/ql/src/gen/vectorization/ExpressionTemplates/TimestampScalarArithmeticDateColumn.txt
@@ -37,6 +37,7 @@ import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.NullUtil;
 import org.apache.hadoop.hive.ql.util.DateTimeMath;
 import org.apache.hadoop.hive.serde2.io.DateWritable;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 /**
  * Generated from template TimestampScalarArithmeticDateColumnBase.txt.
@@ -73,7 +74,7 @@ public class <ClassName> extends VectorExpression {
    *
    * @batch a package of rows with each column stored in a vector
    */
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     // return immediately if batch is empty
     final int n = batch.size;

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/gen/vectorization/ExpressionTemplates/TimestampScalarArithmeticIntervalYearMonthColumn.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/TimestampScalarArithmeticIntervalYearMonthColumn.txt b/ql/src/gen/vectorization/ExpressionTemplates/TimestampScalarArithmeticIntervalYearMonthColumn.txt
index 9db7b53..dc4f5c8 100644
--- a/ql/src/gen/vectorization/ExpressionTemplates/TimestampScalarArithmeticIntervalYearMonthColumn.txt
+++ b/ql/src/gen/vectorization/ExpressionTemplates/TimestampScalarArithmeticIntervalYearMonthColumn.txt
@@ -36,6 +36,7 @@ import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.NullUtil;
 import org.apache.hadoop.hive.ql.util.DateTimeMath;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 /**
  * Generated from template TimestampScalarArithmeticIntervalYearMonthColumn.txt.
@@ -72,7 +73,7 @@ public class <ClassName> extends VectorExpression {
    *
    * @batch a package of rows with each column stored in a vector
    */
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     // return immediately if batch is empty
     final int n = batch.size;

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/gen/vectorization/ExpressionTemplates/TimestampScalarArithmeticTimestampColumn.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/TimestampScalarArithmeticTimestampColumn.txt b/ql/src/gen/vectorization/ExpressionTemplates/TimestampScalarArithmeticTimestampColumn.txt
index e860e4d..1b1117e 100644
--- a/ql/src/gen/vectorization/ExpressionTemplates/TimestampScalarArithmeticTimestampColumn.txt
+++ b/ql/src/gen/vectorization/ExpressionTemplates/TimestampScalarArithmeticTimestampColumn.txt
@@ -35,6 +35,7 @@ import org.apache.hadoop.hive.ql.exec.vector.TimestampColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.NullUtil;
 import org.apache.hadoop.hive.ql.util.DateTimeMath;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 /**
  * Generated from template TimestampScalarArithmeticTimestampColumnBase.txt.
@@ -70,7 +71,7 @@ public class <ClassName> extends VectorExpression {
    *
    * @batch a package of rows with each column stored in a vector
    */
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     // return immediately if batch is empty
     final int n = batch.size;

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/gen/vectorization/ExpressionTemplates/TimestampScalarCompareTimestampColumn.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/TimestampScalarCompareTimestampColumn.txt b/ql/src/gen/vectorization/ExpressionTemplates/TimestampScalarCompareTimestampColumn.txt
index 10f6162..c409a6b 100644
--- a/ql/src/gen/vectorization/ExpressionTemplates/TimestampScalarCompareTimestampColumn.txt
+++ b/ql/src/gen/vectorization/ExpressionTemplates/TimestampScalarCompareTimestampColumn.txt
@@ -29,6 +29,7 @@ import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.*;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 /**
  * Generated from template ScalarCompareTimestamp.txt, which covers comparison
@@ -57,7 +58,7 @@ public class <ClassName> extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     // return immediately if batch is empty
     final int n = batch.size;

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/gen/vectorization/TestTemplates/TestClass.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/TestTemplates/TestClass.txt b/ql/src/gen/vectorization/TestTemplates/TestClass.txt
index f15695a..8f02195 100644
--- a/ql/src/gen/vectorization/TestTemplates/TestClass.txt
+++ b/ql/src/gen/vectorization/TestTemplates/TestClass.txt
@@ -26,7 +26,7 @@ import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.util.VectorizedRowGroupGenUtil;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
 import org.junit.Test;
-
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 /**
  *

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/gen/vectorization/TestTemplates/TestColumnColumnFilterVectorExpressionEvaluation.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/TestTemplates/TestColumnColumnFilterVectorExpressionEvaluation.txt b/ql/src/gen/vectorization/TestTemplates/TestColumnColumnFilterVectorExpressionEvaluation.txt
index 2bb1aa3..0fb089a 100644
--- a/ql/src/gen/vectorization/TestTemplates/TestColumnColumnFilterVectorExpressionEvaluation.txt
+++ b/ql/src/gen/vectorization/TestTemplates/TestColumnColumnFilterVectorExpressionEvaluation.txt
@@ -15,7 +15,7 @@
    limitations under the License.
 -->
   @Test
-  public void <TestName>() {
+  public void <TestName>() throws HiveException {
 
     Random rand = new Random(SEED);
 

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/gen/vectorization/TestTemplates/TestColumnColumnOperationVectorExpressionCheckedEvaluation.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/TestTemplates/TestColumnColumnOperationVectorExpressionCheckedEvaluation.txt b/ql/src/gen/vectorization/TestTemplates/TestColumnColumnOperationVectorExpressionCheckedEvaluation.txt
index 069d9ab..3b926b9 100644
--- a/ql/src/gen/vectorization/TestTemplates/TestColumnColumnOperationVectorExpressionCheckedEvaluation.txt
+++ b/ql/src/gen/vectorization/TestTemplates/TestColumnColumnOperationVectorExpressionCheckedEvaluation.txt
@@ -15,7 +15,7 @@
    limitations under the License.
 -->
   @Test
-  public void <TestName>() {
+  public void <TestName>() throws HiveException {
 
     Random rand = new Random(SEED);
 

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/gen/vectorization/TestTemplates/TestColumnColumnOperationVectorExpressionEvaluation.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/TestTemplates/TestColumnColumnOperationVectorExpressionEvaluation.txt b/ql/src/gen/vectorization/TestTemplates/TestColumnColumnOperationVectorExpressionEvaluation.txt
index 3c8f8822..a0cf298 100644
--- a/ql/src/gen/vectorization/TestTemplates/TestColumnColumnOperationVectorExpressionEvaluation.txt
+++ b/ql/src/gen/vectorization/TestTemplates/TestColumnColumnOperationVectorExpressionEvaluation.txt
@@ -15,7 +15,7 @@
    limitations under the License.
 -->
   @Test
-  public void <TestName>() {
+  public void <TestName>() throws HiveException {
 
     Random rand = new Random(SEED);
 

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/gen/vectorization/TestTemplates/TestColumnScalarFilterVectorExpressionEvaluation.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/TestTemplates/TestColumnScalarFilterVectorExpressionEvaluation.txt b/ql/src/gen/vectorization/TestTemplates/TestColumnScalarFilterVectorExpressionEvaluation.txt
index 290c94c..479540a 100644
--- a/ql/src/gen/vectorization/TestTemplates/TestColumnScalarFilterVectorExpressionEvaluation.txt
+++ b/ql/src/gen/vectorization/TestTemplates/TestColumnScalarFilterVectorExpressionEvaluation.txt
@@ -15,7 +15,7 @@
    limitations under the License.
 -->
   @Test
-  public void <TestName>() {
+  public void <TestName>() throws HiveException {
 
     Random rand = new Random(SEED);
 

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/gen/vectorization/TestTemplates/TestColumnScalarOperationVectorExpressionCheckedEvaluation.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/TestTemplates/TestColumnScalarOperationVectorExpressionCheckedEvaluation.txt b/ql/src/gen/vectorization/TestTemplates/TestColumnScalarOperationVectorExpressionCheckedEvaluation.txt
index c918f3b..2095971 100644
--- a/ql/src/gen/vectorization/TestTemplates/TestColumnScalarOperationVectorExpressionCheckedEvaluation.txt
+++ b/ql/src/gen/vectorization/TestTemplates/TestColumnScalarOperationVectorExpressionCheckedEvaluation.txt
@@ -15,7 +15,7 @@
    limitations under the License.
 -->
   @Test
-  public void <TestName>() {
+  public void <TestName>() throws HiveException {
 
     Random rand = new Random(SEED);
 

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/gen/vectorization/TestTemplates/TestColumnScalarOperationVectorExpressionEvaluation.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/TestTemplates/TestColumnScalarOperationVectorExpressionEvaluation.txt b/ql/src/gen/vectorization/TestTemplates/TestColumnScalarOperationVectorExpressionEvaluation.txt
index 991135c..04873a5 100644
--- a/ql/src/gen/vectorization/TestTemplates/TestColumnScalarOperationVectorExpressionEvaluation.txt
+++ b/ql/src/gen/vectorization/TestTemplates/TestColumnScalarOperationVectorExpressionEvaluation.txt
@@ -15,7 +15,7 @@
    limitations under the License.
 -->
   @Test
-  public void <TestName>() {
+  public void <TestName>() throws HiveException {
 
     Random rand = new Random(SEED);
 

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/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 abbbe9a..2cad04b 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
@@ -154,6 +154,8 @@ public class VectorizationContext {
   private boolean reuseScratchColumns =
       HiveConf.ConfVars.HIVE_VECTORIZATION_TESTING_REUSE_SCRATCH_COLUMNS.defaultBoolVal;
 
+  private boolean adaptorSuppressEvaluateExceptions;
+
   private void setHiveConfVars(HiveConf hiveConf) {
     hiveVectorAdaptorUsageMode = HiveVectorAdaptorUsageMode.getHiveConfValue(hiveConf);
     hiveVectorIfStmtMode = HiveVectorIfStmtMode.getHiveConfValue(hiveConf);
@@ -162,6 +164,9 @@ public class VectorizationContext {
     this.ocm.setReuseColumns(reuseScratchColumns);
     useCheckedVectorExpressions =
         HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.HIVE_VECTORIZATION_USE_CHECKED_EXPRESSIONS);
+    adaptorSuppressEvaluateExceptions =
+        HiveConf.getBoolVar(
+            hiveConf, HiveConf.ConfVars.HIVE_VECTORIZED_ADAPTOR_SUPPRESS_EVALUATE_EXCEPTIONS);
   }
 
   private void copyHiveConfVars(VectorizationContext vContextEnvironment) {
@@ -3360,7 +3365,8 @@ public class VectorizationContext {
     final int outputColumnNum = ocm.allocateOutputColumn(expr.getTypeInfo());
 
     // Make vectorized operator
-    VectorExpression ve = new VectorUDFAdaptor(expr, outputColumnNum, resultTypeName, argDescs);
+    VectorUDFAdaptor ve = new VectorUDFAdaptor(expr, outputColumnNum, resultTypeName, argDescs);
+    ve.setSuppressEvaluateExceptions(adaptorSuppressEvaluateExceptions);
 
     // Set child expressions
     VectorExpression[] childVEs = null;

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/AbstractFilterStringColLikeStringScalar.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/AbstractFilterStringColLikeStringScalar.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/AbstractFilterStringColLikeStringScalar.java
index 688bc1f..277f866 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/AbstractFilterStringColLikeStringScalar.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/AbstractFilterStringColLikeStringScalar.java
@@ -90,7 +90,7 @@ public abstract class AbstractFilterStringColLikeStringScalar extends VectorExpr
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     if (childExpressions != null) {
       super.evaluateChildren(batch);

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDateToTimestamp.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDateToTimestamp.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDateToTimestamp.java
index f1a584e..dbd7c01 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDateToTimestamp.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDateToTimestamp.java
@@ -24,6 +24,7 @@ import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.TimestampColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.serde2.io.DateWritable;
 
 public class CastDateToTimestamp extends VectorExpression {
@@ -49,7 +50,7 @@ public class CastDateToTimestamp extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     if (childExpressions != null) {
       this.evaluateChildren(batch);

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDecimalToDecimal.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDecimalToDecimal.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDecimalToDecimal.java
index 8326002..5e0d570 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDecimalToDecimal.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDecimalToDecimal.java
@@ -23,6 +23,7 @@ import java.util.Arrays;
 import org.apache.hadoop.hive.ql.exec.vector.DecimalColumnVector;
 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;
 
 /**
  * Cast a decimal to a decimal, accounting for precision and scale changes.
@@ -66,7 +67,7 @@ public class CastDecimalToDecimal extends VectorExpression {
    * respectively.
    */
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     if (childExpressions != null) {
       super.evaluateChildren(batch);

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDecimalToTimestamp.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDecimalToTimestamp.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDecimalToTimestamp.java
index 8107c44..c7c397b 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDecimalToTimestamp.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDecimalToTimestamp.java
@@ -20,10 +20,8 @@ package org.apache.hadoop.hive.ql.exec.vector.expressions;
 
 import java.sql.Timestamp;
 
-import org.apache.hadoop.hive.common.type.HiveDecimal;
 import org.apache.hadoop.hive.ql.exec.vector.DecimalColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.TimestampColumnVector;
-import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.util.TimestampUtils;
 import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
 

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDoubleToTimestamp.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDoubleToTimestamp.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDoubleToTimestamp.java
index 5494579..ad00987 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDoubleToTimestamp.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDoubleToTimestamp.java
@@ -24,6 +24,7 @@ import org.apache.hadoop.hive.ql.exec.vector.DoubleColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.TimestampColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.util.TimestampUtils;
 
 public class CastDoubleToTimestamp extends VectorExpression {
@@ -47,7 +48,7 @@ public class CastDoubleToTimestamp extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     if (childExpressions != null) {
       this.evaluateChildren(batch);

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastLongToDate.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastLongToDate.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastLongToDate.java
index 47b6556..b2185d9 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastLongToDate.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastLongToDate.java
@@ -21,6 +21,7 @@ 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.VectorExpressionDescriptor;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.serde2.io.DateWritable;
 import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector.PrimitiveCategory;
 import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo;
@@ -46,7 +47,7 @@ public class CastLongToDate extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     if (childExpressions != null) {
       super.evaluateChildren(batch);

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastLongToTimestamp.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastLongToTimestamp.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastLongToTimestamp.java
index a3c4212..9e0c00e 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastLongToTimestamp.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastLongToTimestamp.java
@@ -22,6 +22,7 @@ import java.util.Arrays;
 
 import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
 import org.apache.hadoop.hive.ql.exec.vector.*;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.serde2.io.TimestampWritable;
 
 public class CastLongToTimestamp extends VectorExpression {
@@ -47,7 +48,7 @@ public class CastLongToTimestamp extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     if (childExpressions != null) {
       this.evaluateChildren(batch);

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastMillisecondsLongToTimestamp.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastMillisecondsLongToTimestamp.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastMillisecondsLongToTimestamp.java
index 6a29c62..a9ede6b 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastMillisecondsLongToTimestamp.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastMillisecondsLongToTimestamp.java
@@ -22,6 +22,7 @@ import java.util.Arrays;
 
 import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
 import org.apache.hadoop.hive.ql.exec.vector.*;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.serde2.io.TimestampWritable;
 
 public class CastMillisecondsLongToTimestamp extends VectorExpression {
@@ -45,7 +46,7 @@ public class CastMillisecondsLongToTimestamp extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     if (childExpressions != null) {
       this.evaluateChildren(batch);

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastStringToDate.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastStringToDate.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastStringToDate.java
index b55712a..6edd7b9 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastStringToDate.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastStringToDate.java
@@ -54,7 +54,7 @@ public class CastStringToDate extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     if (childExpressions != null) {
       super.evaluateChildren(batch);

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastStringToDecimal.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastStringToDecimal.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastStringToDecimal.java
index cbefa80..d8d7dae 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastStringToDecimal.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastStringToDecimal.java
@@ -25,6 +25,7 @@ import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.DecimalColumnVector;
 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;
 
 /**
  * Cast a string to a decimal.
@@ -72,7 +73,7 @@ public class CastStringToDecimal extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     if (childExpressions != null) {
       super.evaluateChildren(batch);

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastStringToDouble.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastStringToDouble.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastStringToDouble.java
index 9ad442a..f4020a4 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastStringToDouble.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastStringToDouble.java
@@ -24,6 +24,7 @@ import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.DoubleColumnVector;
 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;
 import org.apache.hadoop.hive.serde2.lazy.LazyUtils;
 import org.apache.hadoop.hive.serde2.lazy.fast.StringToDouble;
 
@@ -76,7 +77,7 @@ public class CastStringToDouble extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     if (childExpressions != null) {
       super.evaluateChildren(batch);

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastStringToIntervalDayTime.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastStringToIntervalDayTime.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastStringToIntervalDayTime.java
index 8a64dcf..790328d 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastStringToIntervalDayTime.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastStringToIntervalDayTime.java
@@ -26,6 +26,7 @@ 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;
 import org.apache.hadoop.hive.serde.serdeConstants;
 import org.apache.hive.common.util.DateUtils;
 
@@ -51,7 +52,7 @@ public class CastStringToIntervalDayTime extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     if (childExpressions != null) {
       super.evaluateChildren(batch);

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastStringToIntervalYearMonth.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastStringToIntervalYearMonth.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastStringToIntervalYearMonth.java
index 598113f..4fd0859 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastStringToIntervalYearMonth.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastStringToIntervalYearMonth.java
@@ -25,6 +25,7 @@ import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
 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;
 import org.apache.hadoop.hive.serde.serdeConstants;
 
 
@@ -49,7 +50,7 @@ public class CastStringToIntervalYearMonth extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     if (childExpressions != null) {
       super.evaluateChildren(batch);

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastStringToLong.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastStringToLong.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastStringToLong.java
index e3da77e..58235de 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastStringToLong.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastStringToLong.java
@@ -174,7 +174,7 @@ public class CastStringToLong extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     if (childExpressions != null) {
       super.evaluateChildren(batch);

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastTimestampToBoolean.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastTimestampToBoolean.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastTimestampToBoolean.java
index 1836131..c257574 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastTimestampToBoolean.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastTimestampToBoolean.java
@@ -23,6 +23,7 @@ import java.util.Arrays;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.MathExpr;
 import org.apache.hadoop.hive.ql.exec.vector.*;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 public class CastTimestampToBoolean extends VectorExpression {
   private static final long serialVersionUID = 1L;
@@ -47,7 +48,7 @@ public class CastTimestampToBoolean extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     if (childExpressions != null) {
       this.evaluateChildren(batch);

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastTimestampToDouble.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastTimestampToDouble.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastTimestampToDouble.java
index c11797b..eedde7a 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastTimestampToDouble.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastTimestampToDouble.java
@@ -22,6 +22,7 @@ import java.util.Arrays;
 
 import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
 import org.apache.hadoop.hive.ql.exec.vector.*;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.serde2.io.TimestampWritable;
 
 public class CastTimestampToDouble extends VectorExpression {
@@ -42,7 +43,7 @@ public class CastTimestampToDouble extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     if (childExpressions != null) {
       this.evaluateChildren(batch);

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastTimestampToLong.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastTimestampToLong.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastTimestampToLong.java
index a0f0927..42e005e 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastTimestampToLong.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastTimestampToLong.java
@@ -23,6 +23,7 @@ import java.util.Arrays;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.MathExpr;
 import org.apache.hadoop.hive.ql.exec.vector.*;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 public class CastTimestampToLong extends VectorExpression {
   private static final long serialVersionUID = 1L;
@@ -39,7 +40,7 @@ public class CastTimestampToLong extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     if (childExpressions != null) {
       this.evaluateChildren(batch);

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/ColAndCol.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/ColAndCol.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/ColAndCol.java
index 6fb29a8..c6b52fa 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/ColAndCol.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/ColAndCol.java
@@ -22,6 +22,7 @@ import java.util.Arrays;
 import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
 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;
 
 /**
  * Evaluate AND of two boolean columns and store result in the output boolean column.
@@ -48,7 +49,7 @@ public class ColAndCol extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     if (childExpressions != null) {
       super.evaluateChildren(batch);

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/ColOrCol.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/ColOrCol.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/ColOrCol.java
index 9208cd4..6d816d1 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/ColOrCol.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/ColOrCol.java
@@ -23,6 +23,7 @@ import java.util.Arrays;
 import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
 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;
 
 /**
  * This class performs OR expression on two input columns and stores,
@@ -51,7 +52,7 @@ public class ColOrCol extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     if (childExpressions != null) {
       super.evaluateChildren(batch);

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/DateColSubtractDateColumn.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/DateColSubtractDateColumn.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/DateColSubtractDateColumn.java
index 7342d9e..d963b87 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/DateColSubtractDateColumn.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/DateColSubtractDateColumn.java
@@ -57,7 +57,7 @@ public class DateColSubtractDateColumn extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     if (childExpressions != null) {
       super.evaluateChildren(batch);

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/DateColSubtractDateScalar.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/DateColSubtractDateScalar.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/DateColSubtractDateScalar.java
index 3ea189a..8942b78 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/DateColSubtractDateScalar.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/DateColSubtractDateScalar.java
@@ -59,7 +59,7 @@ public class DateColSubtractDateScalar extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     if (childExpressions != null) {
       super.evaluateChildren(batch);

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/DateScalarSubtractDateColumn.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/DateScalarSubtractDateColumn.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/DateScalarSubtractDateColumn.java
index a87ae39..68b038f 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/DateScalarSubtractDateColumn.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/DateScalarSubtractDateColumn.java
@@ -61,7 +61,7 @@ public class DateScalarSubtractDateColumn extends VectorExpression {
    *
    * @batch a package of rows with each column stored in a vector
    */
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     if (childExpressions != null) {
       super.evaluateChildren(batch);

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/DecimalColumnInList.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/DecimalColumnInList.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/DecimalColumnInList.java
index a677f90..e150789 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/DecimalColumnInList.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/DecimalColumnInList.java
@@ -70,7 +70,7 @@ public class DecimalColumnInList extends VectorExpression implements IDecimalInE
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     if (childExpressions != null) {
       super.evaluateChildren(batch);

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/DecimalToStringUnaryUDF.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/DecimalToStringUnaryUDF.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/DecimalToStringUnaryUDF.java
index 452bd5e..84cb6c8 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/DecimalToStringUnaryUDF.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/DecimalToStringUnaryUDF.java
@@ -24,6 +24,7 @@ import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.DecimalColumnVector;
 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;
 
 /**
  * This is a superclass for unary decimal functions returning strings that operate directly on the
@@ -48,7 +49,7 @@ abstract public class DecimalToStringUnaryUDF extends VectorExpression {
   abstract protected void func(BytesColumnVector outputColVector, DecimalColumnVector inputColVector, int i);
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     if (childExpressions != null) {
       super.evaluateChildren(batch);

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/DoubleColumnInList.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/DoubleColumnInList.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/DoubleColumnInList.java
index 89e58f1..20458b2 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/DoubleColumnInList.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/DoubleColumnInList.java
@@ -25,6 +25,7 @@ import org.apache.hadoop.hive.ql.exec.vector.DoubleColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 /**
  * Output a boolean value indicating if a column is IN a list of constants.
@@ -51,7 +52,7 @@ public class DoubleColumnInList extends VectorExpression implements IDoubleInExp
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     if (childExpressions != null) {
       super.evaluateChildren(batch);

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/DoubleToStringUnaryUDF.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/DoubleToStringUnaryUDF.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/DoubleToStringUnaryUDF.java
index b33046e..00d2710 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/DoubleToStringUnaryUDF.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/DoubleToStringUnaryUDF.java
@@ -24,6 +24,7 @@ import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.DoubleColumnVector;
 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;
 
 /**
  * This is a superclass for unary double functions returning strings that operate directly on the
@@ -49,7 +50,7 @@ abstract public class DoubleToStringUnaryUDF extends VectorExpression {
   abstract protected void func(BytesColumnVector outputColVector, double[] vector, int i);
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     if (childExpressions != null) {
       super.evaluateChildren(batch);

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FilterColAndScalar.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FilterColAndScalar.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FilterColAndScalar.java
index 17922d9..7c78e58 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FilterColAndScalar.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FilterColAndScalar.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.hive.ql.exec.vector.expressions;
 
 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;
 
 public class FilterColAndScalar extends VectorExpression {
 
@@ -42,7 +43,7 @@ public class FilterColAndScalar extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
     if (value == 0) {
       batch.size = 0;
       return;

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FilterColOrScalar.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FilterColOrScalar.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FilterColOrScalar.java
index 6a5a817..4dc9715 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FilterColOrScalar.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FilterColOrScalar.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.hive.ql.exec.vector.expressions;
 
 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;
 
 public class FilterColOrScalar extends VectorExpression {
 
@@ -43,7 +44,7 @@ public class FilterColOrScalar extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
     // Evaluate children only of scalar is FALSE.
     if (value == 0) {
       super.evaluateChildren(batch);

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FilterDecimalColumnInList.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FilterDecimalColumnInList.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FilterDecimalColumnInList.java
index c8d696b..b6ca2ac 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FilterDecimalColumnInList.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FilterDecimalColumnInList.java
@@ -67,7 +67,7 @@ public class FilterDecimalColumnInList extends VectorExpression implements IDeci
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     if (childExpressions != null) {
       super.evaluateChildren(batch);

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FilterDoubleColumnInList.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FilterDoubleColumnInList.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FilterDoubleColumnInList.java
index 35b73c3..c4e0ed6 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FilterDoubleColumnInList.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FilterDoubleColumnInList.java
@@ -71,7 +71,7 @@ public class FilterDoubleColumnInList extends VectorExpression implements IDoubl
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     if (childExpressions != null) {
       super.evaluateChildren(batch);

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FilterExprAndExpr.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FilterExprAndExpr.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FilterExprAndExpr.java
index 52e95f2..9a239b6 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FilterExprAndExpr.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FilterExprAndExpr.java
@@ -22,6 +22,7 @@ import java.util.Arrays;
 
 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;
 
 /**
  * This class represents a non leaf binary operator in the expression tree.
@@ -35,7 +36,7 @@ public class FilterExprAndExpr extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
     childExpressions[0].evaluate(batch);
     for (int childIndex = 1; childIndex < childExpressions.length; childIndex++) {
       childExpressions[childIndex].evaluate(batch);

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FilterExprOrExpr.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FilterExprOrExpr.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FilterExprOrExpr.java
index 5593d48..a32e394 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FilterExprOrExpr.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FilterExprOrExpr.java
@@ -22,6 +22,7 @@ import com.google.common.base.Preconditions;
 
 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;
 
 /**
  * This class represents an Or expression. This applies short circuit optimization.
@@ -113,7 +114,7 @@ public class FilterExprOrExpr extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
     int n = batch.size;
     if (n <= 0) {
       return;

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FilterLongColumnInList.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FilterLongColumnInList.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FilterLongColumnInList.java
index 0c6b0f0..312a388 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FilterLongColumnInList.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FilterLongColumnInList.java
@@ -69,7 +69,7 @@ public class FilterLongColumnInList extends VectorExpression implements ILongInE
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     if (childExpressions != null) {
       super.evaluateChildren(batch);

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FilterScalarAndColumn.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FilterScalarAndColumn.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FilterScalarAndColumn.java
index 9b0a015..9b319e5 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FilterScalarAndColumn.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FilterScalarAndColumn.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.hive.ql.exec.vector.expressions;
 
 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;
 
 public class FilterScalarAndColumn extends VectorExpression {
 
@@ -42,7 +43,7 @@ public class FilterScalarAndColumn extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
     if (value == 0) {
       batch.size = 0;
       return;

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FilterScalarOrColumn.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FilterScalarOrColumn.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FilterScalarOrColumn.java
index 355377c..6e12404 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FilterScalarOrColumn.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FilterScalarOrColumn.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.hive.ql.exec.vector.expressions;
 
 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;
 
 public class FilterScalarOrColumn extends VectorExpression {
 
@@ -42,7 +43,7 @@ public class FilterScalarOrColumn extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
     // Evaluate children only of scalar is FALSE.
     if (value == 0) {
       super.evaluateChildren(batch);

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FilterStringColumnInList.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FilterStringColumnInList.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FilterStringColumnInList.java
index 1bc79fc..3456abb 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FilterStringColumnInList.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FilterStringColumnInList.java
@@ -24,6 +24,7 @@ import java.util.Arrays;
 import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor.Descriptor;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 /**
  * Evaluate an IN filter on a batch for a vector of strings.
@@ -54,7 +55,7 @@ public class FilterStringColumnInList extends VectorExpression implements IStrin
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     if (childExpressions != null) {
       super.evaluateChildren(batch);

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FilterStructColumnInList.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FilterStructColumnInList.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FilterStructColumnInList.java
index e4e0665..ca1bf42 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FilterStructColumnInList.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FilterStructColumnInList.java
@@ -61,7 +61,7 @@ public class FilterStructColumnInList extends FilterStringColumnInList implement
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     final int logicalSize = batch.size;
     if (logicalSize == 0) {

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FilterTimestampColumnInList.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FilterTimestampColumnInList.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FilterTimestampColumnInList.java
index ca92c7c..cb22e2a 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FilterTimestampColumnInList.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FilterTimestampColumnInList.java
@@ -65,7 +65,7 @@ public class FilterTimestampColumnInList extends VectorExpression implements ITi
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     if (childExpressions != null) {
       super.evaluateChildren(batch);


[27/50] [abbrv] hive git commit: HIVE-19479 : encoded stream seek is incorrect for 0-length RGs in LLAP IO (Sergey Shelukhin, reviewed by Prasanth Jayachandran)

Posted by vg...@apache.org.
HIVE-19479 : encoded stream seek is incorrect for 0-length RGs in LLAP IO (Sergey Shelukhin, reviewed by Prasanth Jayachandran)


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

Branch: refs/heads/branch-3.0.0
Commit: e941bea80393e74efd64b07355b2e0fac384f7cc
Parents: 9ebb2ff
Author: sergey <se...@apache.org>
Authored: Fri May 11 12:01:10 2018 -0700
Committer: Vineet Garg <vg...@apache.org>
Committed: Sun May 13 21:03:44 2018 -0700

----------------------------------------------------------------------
 .../llap/io/decode/OrcEncodedDataConsumer.java  |   3 +
 .../ql/io/orc/encoded/EncodedReaderImpl.java    |  10 +-
 .../orc/encoded/EncodedTreeReaderFactory.java   | 118 ++++++++++---------
 3 files changed, 70 insertions(+), 61 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/e941bea8/llap-server/src/java/org/apache/hadoop/hive/llap/io/decode/OrcEncodedDataConsumer.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/io/decode/OrcEncodedDataConsumer.java b/llap-server/src/java/org/apache/hadoop/hive/llap/io/decode/OrcEncodedDataConsumer.java
index fc0c66a..05282db 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/io/decode/OrcEncodedDataConsumer.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/io/decode/OrcEncodedDataConsumer.java
@@ -296,6 +296,9 @@ public class OrcEncodedDataConsumer
       ConsumerStripeMetadata stripeMetadata) throws IOException {
     PositionProvider[] pps = createPositionProviders(
         columnReaders, batch.getBatchKey(), stripeMetadata);
+    if (LlapIoImpl.ORC_LOGGER.isTraceEnabled()) {
+      LlapIoImpl.ORC_LOGGER.trace("Created pps {}", Arrays.toString(pps));
+    }
     if (pps == null) return;
     for (int i = 0; i < columnReaders.length; i++) {
       TreeReader reader = columnReaders[i];

http://git-wip-us.apache.org/repos/asf/hive/blob/e941bea8/ql/src/java/org/apache/hadoop/hive/ql/io/orc/encoded/EncodedReaderImpl.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/encoded/EncodedReaderImpl.java b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/encoded/EncodedReaderImpl.java
index 1d7eceb..348f9df 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/encoded/EncodedReaderImpl.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/encoded/EncodedReaderImpl.java
@@ -435,12 +435,12 @@ class EncodedReaderImpl implements EncodedReader {
               try {
                 if (RecordReaderUtils.isDictionary(sctx.kind, ctx.encoding) || index == null) {
                   // This stream is for entire stripe and needed for every RG; uncompress once and reuse.
-                  if (isTracingEnabled) {
-                    LOG.trace("Getting stripe-level stream [" + sctx.kind + ", " + ctx.encoding + "] for"
-                        + " column " + ctx.colIx + " RG " + rgIx + " at " + sctx.offset + ", " + sctx.length);
-                  }
-                  trace.logStartStripeStream(sctx.kind);
                   if (sctx.stripeLevelStream == null) {
+                    if (isTracingEnabled) {
+                      LOG.trace("Getting stripe-level stream [" + sctx.kind + ", " + ctx.encoding + "] for"
+                          + " column " + ctx.colIx + " RG " + rgIx + " at " + sctx.offset + ", " + sctx.length);
+                    }
+                    trace.logStartStripeStream(sctx.kind);
                     sctx.stripeLevelStream = POOLS.csdPool.take();
                     // We will be using this for each RG while also sending RGs to processing.
                     // To avoid buffers being unlocked, run refcount one ahead; so each RG 

http://git-wip-us.apache.org/repos/asf/hive/blob/e941bea8/ql/src/java/org/apache/hadoop/hive/ql/io/orc/encoded/EncodedTreeReaderFactory.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/encoded/EncodedTreeReaderFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/encoded/EncodedTreeReaderFactory.java
index 42532f9..646b214 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/encoded/EncodedTreeReaderFactory.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/encoded/EncodedTreeReaderFactory.java
@@ -31,6 +31,7 @@ import org.apache.hadoop.hive.ql.io.orc.encoded.Reader.OrcEncodedColumnBatch;
 import org.apache.orc.CompressionCodec;
 import org.apache.orc.TypeDescription;
 import org.apache.orc.TypeDescription.Category;
+import org.apache.orc.impl.InStream;
 import org.apache.orc.impl.PositionProvider;
 import org.apache.orc.impl.SettableUncompressedStream;
 import org.apache.orc.impl.TreeReaderFactory;
@@ -213,6 +214,11 @@ public class EncodedTreeReaderFactory extends TreeReaderFactory {
     }
   }
 
+  private static void skipCompressedIndex(boolean isCompressed, PositionProvider index) {
+    if (!isCompressed) return;
+    index.getNext();
+  }
+
   protected static class StringStreamReader extends StringTreeReader
       implements SettableTreeReader {
     private boolean _isFileCompressed;
@@ -260,30 +266,30 @@ public class EncodedTreeReaderFactory extends TreeReaderFactory {
 
         // data stream could be empty stream or already reached end of stream before present stream.
         // This can happen if all values in stream are nulls or last row group values are all null.
+        skipCompressedIndex(_isFileCompressed, index);
         if (_dataStream != null && _dataStream.available() > 0) {
-          if (_isFileCompressed) {
-            index.getNext();
-          }
           ((StringDictionaryTreeReader) reader).getReader().seek(index);
-        }
+        } // No need to skip seek here, index won't be used anymore.
       } else {
         // DIRECT encoding
 
         // data stream could be empty stream or already reached end of stream before present stream.
         // This can happen if all values in stream are nulls or last row group values are all null.
+        skipCompressedIndex(_isFileCompressed, index);
+        // TODO: why does the original code not just use _dataStream that it passes in as stream?
+        InStream stream = ((StringDirectTreeReader) reader).getStream();
+        // TODO: not clear why this check and skipSeek are needed.
         if (_dataStream != null && _dataStream.available() > 0) {
-          if (_isFileCompressed) {
-            index.getNext();
-          }
-          ((StringDirectTreeReader) reader).getStream().seek(index);
+          stream.seek(index);
+        } else {
+          assert stream == _dataStream;
+          skipSeek(index);
         }
 
+        skipCompressedIndex(_isFileCompressed, index);
         if (_lengthStream != null && _lengthStream.available() > 0) {
-          if (_isFileCompressed) {
-            index.getNext();
-          }
           ((StringDirectTreeReader) reader).getLengths().seek(index);
-        }
+        } // No need to skip seek here, index won't be used anymore.
       }
     }
 
@@ -830,10 +836,8 @@ public class EncodedTreeReaderFactory extends TreeReaderFactory {
 
       // data stream could be empty stream or already reached end of stream before present stream.
       // This can happen if all values in stream are nulls or last row group values are all null.
+      skipCompressedIndex(_isFileCompressed, index);
       if (_dataStream.available() > 0) {
-        if (_isFileCompressed) {
-          index.getNext();
-        }
         stream.seek(index);
       }
     }
@@ -945,10 +949,8 @@ public class EncodedTreeReaderFactory extends TreeReaderFactory {
 
       // data stream could be empty stream or already reached end of stream before present stream.
       // This can happen if all values in stream are nulls or last row group values are all null.
+      skipCompressedIndex(_isFileCompressed, index);
       if (_dataStream.available() > 0) {
-        if (_isFileCompressed) {
-          index.getNext();
-        }
         stream.seek(index);
       }
     }
@@ -1071,19 +1073,19 @@ public class EncodedTreeReaderFactory extends TreeReaderFactory {
 
       // data stream could be empty stream or already reached end of stream before present stream.
       // This can happen if all values in stream are nulls or last row group values are all null.
+      skipCompressedIndex(_isFileCompressed, index);
+      // TODO: not clear why this check and skipSeek are needed.
       if (_valueStream.available() > 0) {
-        if (_isFileCompressed) {
-          index.getNext();
-        }
         valueStream.seek(index);
+      } else {
+        assert valueStream == _valueStream;
+        skipSeek(index);
       }
 
+      skipCompressedIndex(_isFileCompressed, index);
       if (_scaleStream.available() > 0) {
-        if (_isFileCompressed) {
-          index.getNext();
-        }
         scaleReader.seek(index);
-      }
+      } // No need to skip seek here, index won't be used anymore.
     }
 
     @Override
@@ -1375,30 +1377,29 @@ public class EncodedTreeReaderFactory extends TreeReaderFactory {
 
         // data stream could be empty stream or already reached end of stream before present stream.
         // This can happen if all values in stream are nulls or last row group values are all null.
+        skipCompressedIndex(_isFileCompressed, index);
         if (_dataStream.available() > 0) {
-          if (_isFileCompressed) {
-            index.getNext();
-          }
           ((StringDictionaryTreeReader) reader).getReader().seek(index);
-        }
+        } // No need to skip seek here, index won't be used anymore.
       } else {
         // DIRECT encoding
 
         // data stream could be empty stream or already reached end of stream before present stream.
         // This can happen if all values in stream are nulls or last row group values are all null.
+        skipCompressedIndex(_isFileCompressed, index);
+        InStream stream = ((StringDirectTreeReader) reader).getStream();
+        // TODO: not clear why this check and skipSeek are needed.
         if (_dataStream.available() > 0) {
-          if (_isFileCompressed) {
-            index.getNext();
-          }
-          ((StringDirectTreeReader) reader).getStream().seek(index);
+          stream.seek(index);
+        } else {
+          assert stream == _dataStream;
+          skipSeek(index);
         }
 
+        skipCompressedIndex(_isFileCompressed, index);
         if (_lengthStream.available() > 0) {
-          if (_isFileCompressed) {
-            index.getNext();
-          }
           ((StringDirectTreeReader) reader).getLengths().seek(index);
-        }
+        } // No need to skip seek here, index won't be used anymore.
       }
     }
 
@@ -1574,30 +1575,29 @@ public class EncodedTreeReaderFactory extends TreeReaderFactory {
 
         // data stream could be empty stream or already reached end of stream before present stream.
         // This can happen if all values in stream are nulls or last row group values are all null.
+        skipCompressedIndex(_isFileCompressed, index);
         if (_dataStream.available() > 0) {
-          if (_isFileCompressed) {
-            index.getNext();
-          }
           ((StringDictionaryTreeReader) reader).getReader().seek(index);
-        }
+        } // No need to skip seek here, index won't be used anymore.
       } else {
         // DIRECT encoding
 
         // data stream could be empty stream or already reached end of stream before present stream.
         // This can happen if all values in stream are nulls or last row group values are all null.
+        skipCompressedIndex(_isFileCompressed, index);
+        InStream stream = ((StringDirectTreeReader) reader).getStream();
+        // TODO: not clear why this check and skipSeek are needed.
         if (_dataStream.available() > 0) {
-          if (_isFileCompressed) {
-            index.getNext();
-          }
-          ((StringDirectTreeReader) reader).getStream().seek(index);
+          stream.seek(index);
+        } else {
+          assert stream == _dataStream;
+          skipSeek(index);
         }
 
+        skipCompressedIndex(_isFileCompressed, index);
         if (_lengthStream.available() > 0) {
-          if (_isFileCompressed) {
-            index.getNext();
-          }
           ((StringDirectTreeReader) reader).getLengths().seek(index);
-        }
+        } // No need to skip seek here, index won't be used anymore.
       }
     }
 
@@ -1885,19 +1885,19 @@ public class EncodedTreeReaderFactory extends TreeReaderFactory {
 
       // data stream could be empty stream or already reached end of stream before present stream.
       // This can happen if all values in stream are nulls or last row group values are all null.
+      skipCompressedIndex(_isFileCompressed, index);
+      // TODO: not clear why this check and skipSeek are needed.
       if (_dataStream.available() > 0) {
-        if (_isFileCompressed) {
-          index.getNext();
-        }
         stream.seek(index);
+      } else {
+        assert stream == _dataStream;
+        skipSeek(index);
       }
 
+      skipCompressedIndex(_isFileCompressed, index);
       if (lengths != null && _lengthsStream.available() > 0) {
-        if (_isFileCompressed) {
-          index.getNext();
-        }
         lengths.seek(index);
-      }
+      } // No need to skip seek here, index won't be used anymore.
     }
 
     @Override
@@ -2132,6 +2132,12 @@ public class EncodedTreeReaderFactory extends TreeReaderFactory {
   }
 
 
+  private static void skipSeek(PositionProvider index) {
+    // Must be consistent with uncompressed stream seek in ORC. See call site comments.
+    index.getNext();
+  }
+
+
   private static TreeReader createEncodedTreeReader(TypeDescription schema,
       List<OrcProto.ColumnEncoding> encodings, OrcEncodedColumnBatch batch,
       CompressionCodec codec, TreeReaderFactory.Context context) throws IOException {


[20/50] [abbrv] hive git commit: HIVE-19477: Hiveserver2 in http mode not emitting metric default.General.open_connections (Jesus Camacho Rodriguez, reviewed by Vaibhav Gumashta)

Posted by vg...@apache.org.
HIVE-19477: Hiveserver2 in http mode not emitting metric default.General.open_connections (Jesus Camacho Rodriguez, reviewed by Vaibhav Gumashta)


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

Branch: refs/heads/branch-3.0.0
Commit: b3313380c994b90334d06b7ce364bcf24fb0e8d3
Parents: 1a31e49
Author: Jesus Camacho Rodriguez <jc...@apache.org>
Authored: Fri May 11 09:00:51 2018 -0700
Committer: Jesus Camacho Rodriguez <jc...@apache.org>
Committed: Fri May 11 09:21:30 2018 -0700

----------------------------------------------------------------------
 .../cli/thrift/ThriftHttpCLIService.java        | 34 ++++++++++++++++++--
 1 file changed, 32 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/b3313380/service/src/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java b/service/src/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java
index a02f13c..0b3f2c3 100644
--- a/service/src/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java
+++ b/service/src/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java
@@ -23,14 +23,17 @@ import java.util.concurrent.ExecutorService;
 import java.util.concurrent.SynchronousQueue;
 import java.util.concurrent.TimeUnit;
 
+import javax.servlet.ServletContextEvent;
+import javax.servlet.ServletContextListener;
 import javax.ws.rs.HttpMethod;
 
+import org.apache.hadoop.hive.common.metrics.common.Metrics;
+import org.apache.hadoop.hive.common.metrics.common.MetricsConstant;
+import org.apache.hadoop.hive.common.metrics.common.MetricsFactory;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.shims.ShimLoader;
-import org.apache.hadoop.hive.shims.Utils;
 import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.util.Shell;
 import org.apache.hive.service.auth.HiveAuthFactory;
 import org.apache.hive.service.cli.CLIService;
 import org.apache.hive.service.rpc.thrift.TCLIService;
@@ -153,6 +156,33 @@ public class ThriftHttpCLIService extends ThriftCLIService {
         LOG.warn("XSRF filter disabled");
       }
 
+      context.addEventListener(new ServletContextListener() {
+        @Override
+        public void contextInitialized(ServletContextEvent servletContextEvent) {
+          Metrics metrics = MetricsFactory.getInstance();
+          if (metrics != null) {
+            try {
+              metrics.incrementCounter(MetricsConstant.OPEN_CONNECTIONS);
+              metrics.incrementCounter(MetricsConstant.CUMULATIVE_CONNECTION_COUNT);
+            } catch (Exception e) {
+              LOG.warn("Error reporting HS2 open connection operation to Metrics system", e);
+            }
+          }
+        }
+
+        @Override
+        public void contextDestroyed(ServletContextEvent servletContextEvent) {
+          Metrics metrics = MetricsFactory.getInstance();
+          if (metrics != null) {
+            try {
+              metrics.decrementCounter(MetricsConstant.OPEN_CONNECTIONS);
+            } catch (Exception e) {
+              LOG.warn("Error reporting HS2 close connection operation to Metrics system", e);
+            }
+          }
+        }
+      });
+
       final String httpPath = getHttpPath(hiveConf
           .getVar(HiveConf.ConfVars.HIVE_SERVER2_THRIFT_HTTP_PATH));
 


[02/50] [abbrv] hive git commit: HIVE-19357: Vectorization: assert_true HiveException erroneously gets suppressed to NULL (Matt McCline, reviewed by Zoltan Haindrich)

Posted by vg...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/MathFuncDoubleToDouble.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/MathFuncDoubleToDouble.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/MathFuncDoubleToDouble.java
index aad408f..01ab9c8 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/MathFuncDoubleToDouble.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/MathFuncDoubleToDouble.java
@@ -23,6 +23,7 @@ import java.util.Arrays;
 import org.apache.hadoop.hive.ql.exec.vector.DoubleColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 /**
  * Implement vectorized math function that takes a double (and optionally additional
@@ -55,7 +56,7 @@ public abstract class MathFuncDoubleToDouble extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     if (childExpressions != null) {
       this.evaluateChildren(batch);

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/MathFuncLongToDouble.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/MathFuncLongToDouble.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/MathFuncLongToDouble.java
index dcebc24..cae6ba1 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/MathFuncLongToDouble.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/MathFuncLongToDouble.java
@@ -23,6 +23,7 @@ import java.util.Arrays;
 import org.apache.hadoop.hive.ql.exec.vector.DoubleColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 /**
  * Implement vectorized math function that takes a double (and optionally additional
@@ -54,7 +55,7 @@ public abstract class MathFuncLongToDouble extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     if (childExpressions != null) {
       this.evaluateChildren(batch);

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/MathFuncLongToLong.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/MathFuncLongToLong.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/MathFuncLongToLong.java
index e5b6902..b9fbbf2 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/MathFuncLongToLong.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/MathFuncLongToLong.java
@@ -23,6 +23,7 @@ import java.util.Arrays;
 import org.apache.hadoop.hive.ql.exec.vector.DoubleColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 /**
  * Implement vectorized math function that takes a long (and optionally additional
@@ -53,7 +54,7 @@ public abstract class MathFuncLongToLong extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     if (childExpressions != null) {
       this.evaluateChildren(batch);

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/NotCol.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/NotCol.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/NotCol.java
index be69f7f..cfd90e8 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/NotCol.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/NotCol.java
@@ -22,6 +22,7 @@ import java.util.Arrays;
 import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
 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;
 
 /**
  * Evaluates the boolean complement of the input.
@@ -44,7 +45,7 @@ public class NotCol extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     if (childExpressions != null) {
       super.evaluateChildren(batch);

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/OctetLength.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/OctetLength.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/OctetLength.java
index 62873e9..ebea38d 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/OctetLength.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/OctetLength.java
@@ -24,6 +24,7 @@ import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
 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;
 
 public class OctetLength extends VectorExpression {
   private static final long serialVersionUID = 1L;
@@ -44,7 +45,7 @@ public class OctetLength extends VectorExpression {
 
   // Calculate the length of the UTF-8 strings in input vector and place results in output vector.
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     if (childExpressions != null) {
       super.evaluateChildren(batch);

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/SelectColumnIsFalse.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/SelectColumnIsFalse.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/SelectColumnIsFalse.java
index 8dc995e..79b8692 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/SelectColumnIsFalse.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/SelectColumnIsFalse.java
@@ -21,6 +21,7 @@ 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.VectorExpressionDescriptor;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 /**
  * This expression selects a row if the given boolean column is false.
@@ -43,7 +44,7 @@ public class SelectColumnIsFalse extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     if (childExpressions != null) {
       super.evaluateChildren(batch);

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/SelectColumnIsNotNull.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/SelectColumnIsNotNull.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/SelectColumnIsNotNull.java
index cf484b8..6c7d31d 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/SelectColumnIsNotNull.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/SelectColumnIsNotNull.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.hive.ql.exec.vector.expressions;
 import org.apache.hadoop.hive.ql.exec.vector.ColumnVector;
 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;
 
 /**
  * This expression selects a row if the given column is null.
@@ -43,7 +44,7 @@ public class SelectColumnIsNotNull extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     if (childExpressions != null) {
       super.evaluateChildren(batch);

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/SelectColumnIsNull.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/SelectColumnIsNull.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/SelectColumnIsNull.java
index 3b9a55e..59f0c2b 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/SelectColumnIsNull.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/SelectColumnIsNull.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.hive.ql.exec.vector.expressions;
 import org.apache.hadoop.hive.ql.exec.vector.ColumnVector;
 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;
 
 /**
  * This expression selects a row if the given column is null.
@@ -44,7 +45,7 @@ public class SelectColumnIsNull extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
     if (childExpressions != null) {
       super.evaluateChildren(batch);
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/SelectColumnIsTrue.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/SelectColumnIsTrue.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/SelectColumnIsTrue.java
index b2f430d..cff064e 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/SelectColumnIsTrue.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/SelectColumnIsTrue.java
@@ -21,6 +21,7 @@ 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.VectorExpressionDescriptor;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 /**
  * This expression selects a row if the given boolean column is true.
@@ -43,7 +44,7 @@ public class SelectColumnIsTrue extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
     if (childExpressions != null) {
       super.evaluateChildren(batch);
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/SelectStringColLikeStringScalar.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/SelectStringColLikeStringScalar.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/SelectStringColLikeStringScalar.java
index db684c3..185680b 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/SelectStringColLikeStringScalar.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/SelectStringColLikeStringScalar.java
@@ -27,6 +27,7 @@ import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
 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;
 
 public class SelectStringColLikeStringScalar extends VectorExpression {
 
@@ -52,7 +53,7 @@ public class SelectStringColLikeStringScalar extends VectorExpression {
   }
 
   @Override
-	public void evaluate(VectorizedRowBatch batch) {
+	public void evaluate(VectorizedRowBatch batch) throws HiveException {
     if (checker == null) {
       checker = borrowChecker();
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/StringColumnInList.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/StringColumnInList.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/StringColumnInList.java
index eb91321..55c2586 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/StringColumnInList.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/StringColumnInList.java
@@ -60,7 +60,7 @@ public class StringColumnInList extends VectorExpression implements IStringInExp
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     if (childExpressions != null) {
       super.evaluateChildren(batch);

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/StringGroupColConcatStringScalar.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/StringGroupColConcatStringScalar.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/StringGroupColConcatStringScalar.java
index 6c92e39..9194e8b 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/StringGroupColConcatStringScalar.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/StringGroupColConcatStringScalar.java
@@ -24,6 +24,7 @@ import java.util.Arrays;
 import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
 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;
 
 /**
  * Vectorized instruction to concatenate a string column to a scalar and put
@@ -50,7 +51,7 @@ public class StringGroupColConcatStringScalar extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     if (childExpressions != null) {
       super.evaluateChildren(batch);

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/StringGroupConcatColCol.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/StringGroupConcatColCol.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/StringGroupConcatColCol.java
index 6c40a28..1c9433b 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/StringGroupConcatColCol.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/StringGroupConcatColCol.java
@@ -24,6 +24,7 @@ import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.ColumnVector;
 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;
 
 /**
  * Vectorized instruction to concatenate two string columns and put
@@ -50,7 +51,7 @@ public class StringGroupConcatColCol extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     if (childExpressions != null) {
       super.evaluateChildren(batch);

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/StringLength.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/StringLength.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/StringLength.java
index f1fabb7..956fd7b 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/StringLength.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/StringLength.java
@@ -24,6 +24,7 @@ import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
 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;
 
 /**
  * Calculate the length of the strings in the input column vector, and store
@@ -48,7 +49,7 @@ public class StringLength extends VectorExpression {
 
   // Calculate the length of the UTF-8 strings in input vector and place results in output vector.
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     if (childExpressions != null) {
       super.evaluateChildren(batch);

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/StringScalarConcatStringGroupCol.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/StringScalarConcatStringGroupCol.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/StringScalarConcatStringGroupCol.java
index a9f09dd..db679b0 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/StringScalarConcatStringGroupCol.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/StringScalarConcatStringGroupCol.java
@@ -24,6 +24,7 @@ import java.util.Arrays;
 import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
 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;
 
 /**
  * Vectorized instruction to concatenate a scalar to a string column and put
@@ -50,7 +51,7 @@ public class StringScalarConcatStringGroupCol extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     if (childExpressions != null) {
         super.evaluateChildren(batch);

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/StringSubstrColStart.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/StringSubstrColStart.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/StringSubstrColStart.java
index 7c58838..411fc4b 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/StringSubstrColStart.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/StringSubstrColStart.java
@@ -24,6 +24,7 @@ import java.util.Arrays;
 import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
 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;
 
 /**
  * This class provides the implementation of vectorized substring, with a single start index
@@ -120,7 +121,7 @@ public class StringSubstrColStart extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
     if (childExpressions != null) {
       super.evaluateChildren(batch);
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/StringSubstrColStartLen.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/StringSubstrColStartLen.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/StringSubstrColStartLen.java
index 7c5d19a..e28c0a7 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/StringSubstrColStartLen.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/StringSubstrColStartLen.java
@@ -24,6 +24,7 @@ import java.util.Arrays;
 import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
 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;
 
 /**
  * This class provides the implementation of vectorized substring, with a start index and length
@@ -141,7 +142,7 @@ public class StringSubstrColStartLen extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
     if (childExpressions != null) {
       super.evaluateChildren(batch);
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/StringUnaryUDF.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/StringUnaryUDF.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/StringUnaryUDF.java
index 9b7005d..2b89648 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/StringUnaryUDF.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/StringUnaryUDF.java
@@ -23,6 +23,7 @@ import java.util.Arrays;
 import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
 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;
 import org.apache.hadoop.io.Text;
 
 /**
@@ -59,7 +60,7 @@ public class StringUnaryUDF extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     if (childExpressions != null) {
       super.evaluateChildren(batch);

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/StringUnaryUDFDirect.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/StringUnaryUDFDirect.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/StringUnaryUDFDirect.java
index 9462347..5726e89 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/StringUnaryUDFDirect.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/StringUnaryUDFDirect.java
@@ -23,6 +23,7 @@ import java.util.Arrays;
 import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
 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;
 
 /**
  * This is a superclass for unary string functions that operate directly on the
@@ -49,7 +50,7 @@ abstract public class StringUnaryUDFDirect extends VectorExpression {
   abstract protected void func(BytesColumnVector outputColVector, byte[][] vector, int[] start, int[] length, int i);
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     if (childExpressions != null) {
       super.evaluateChildren(batch);

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/StructColumnInList.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/StructColumnInList.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/StructColumnInList.java
index 2c661a8..f07be7a 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/StructColumnInList.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/StructColumnInList.java
@@ -63,7 +63,7 @@ public class StructColumnInList extends StringColumnInList implements IStructInE
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     final int logicalSize = batch.size;
     if (logicalSize == 0) {

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/TimestampColumnInList.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/TimestampColumnInList.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/TimestampColumnInList.java
index 31a0ad1..c361764 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/TimestampColumnInList.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/TimestampColumnInList.java
@@ -26,6 +26,7 @@ import org.apache.hadoop.hive.ql.exec.vector.TimestampColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor.Descriptor;
 import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 /**
  * Output a boolean value indicating if a column is IN a list of constants.
@@ -57,7 +58,7 @@ public class TimestampColumnInList extends VectorExpression implements ITimestam
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     if (childExpressions != null) {
       super.evaluateChildren(batch);

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/TimestampToStringUnaryUDF.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/TimestampToStringUnaryUDF.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/TimestampToStringUnaryUDF.java
index 13abfd3..0d9f9f7 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/TimestampToStringUnaryUDF.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/TimestampToStringUnaryUDF.java
@@ -24,6 +24,7 @@ import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.TimestampColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 /**
  * This is a superclass for unary long functions returning strings that operate directly on the
@@ -49,7 +50,7 @@ abstract public class TimestampToStringUnaryUDF extends VectorExpression {
   abstract protected void func(BytesColumnVector outputColVector, TimestampColumnVector inV, int i);
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     if (childExpressions != null) {
       super.evaluateChildren(batch);

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorCoalesce.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorCoalesce.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorCoalesce.java
index c66beb0..2bd726d 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorCoalesce.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorCoalesce.java
@@ -21,6 +21,7 @@ import org.apache.hadoop.hive.ql.exec.vector.ColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
 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;
 
 import com.google.common.base.Preconditions;
 
@@ -53,7 +54,7 @@ public class VectorCoalesce extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     if (childExpressions != null) {
       super.evaluateChildren(batch);

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorElt.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorElt.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorElt.java
index a30a7df..00e529d 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorElt.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorElt.java
@@ -23,6 +23,7 @@ import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
 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;
 
 /*
  * ELT(index, string, ....) returns the string column/expression value at the specified
@@ -50,7 +51,7 @@ public class VectorElt extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     if (childExpressions != null) {
       super.evaluateChildren(batch);

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorExpression.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorExpression.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorExpression.java
index 3d0ee50..893da15 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorExpression.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorExpression.java
@@ -247,8 +247,9 @@ public abstract class VectorExpression implements Serializable {
   /**
    * This is the primary method to implement expression logic.
    * @param batch
+   * @throws HiveException 
    */
-  public abstract void evaluate(VectorizedRowBatch batch);
+  public abstract void evaluate(VectorizedRowBatch batch) throws HiveException;
 
   public void init(Configuration conf) {
     if (childExpressions != null) {
@@ -264,7 +265,7 @@ public abstract class VectorExpression implements Serializable {
    * Evaluate the child expressions on the given input batch.
    * @param vrg {@link VectorizedRowBatch}
    */
-  final protected void evaluateChildren(VectorizedRowBatch vrg) {
+  final protected void evaluateChildren(VectorizedRowBatch vrg) throws HiveException {
     if (childExpressions != null) {
       for (VectorExpression ve : childExpressions) {
         ve.evaluate(vrg);

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorInBloomFilterColDynamicValue.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorInBloomFilterColDynamicValue.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorInBloomFilterColDynamicValue.java
index 72c2980..d8a3cac 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorInBloomFilterColDynamicValue.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorInBloomFilterColDynamicValue.java
@@ -122,7 +122,7 @@ public class VectorInBloomFilterColDynamicValue extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
     if (childExpressions != null) {
       super.evaluateChildren(batch);
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateAddColCol.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateAddColCol.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateAddColCol.java
index f6e9c8b..fa23d89 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateAddColCol.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateAddColCol.java
@@ -72,7 +72,7 @@ public class VectorUDFDateAddColCol extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     if (childExpressions != null) {
       super.evaluateChildren(batch);

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateAddColScalar.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateAddColScalar.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateAddColScalar.java
index 7bb5c54..a73d2e6 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateAddColScalar.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateAddColScalar.java
@@ -72,7 +72,7 @@ public class VectorUDFDateAddColScalar extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     if (childExpressions != null) {
       super.evaluateChildren(batch);

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateAddScalarCol.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateAddScalarCol.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateAddScalarCol.java
index ecde39b..0d418fd 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateAddScalarCol.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateAddScalarCol.java
@@ -81,7 +81,7 @@ public class VectorUDFDateAddScalarCol extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     if (childExpressions != null) {
       super.evaluateChildren(batch);

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateDiffColCol.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateDiffColCol.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateDiffColCol.java
index 0d794fe..84ee944 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateDiffColCol.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateDiffColCol.java
@@ -71,7 +71,7 @@ public class VectorUDFDateDiffColCol extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     if (childExpressions != null) {
       super.evaluateChildren(batch);

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateDiffColScalar.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateDiffColScalar.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateDiffColScalar.java
index 08c91e2..55af413 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateDiffColScalar.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateDiffColScalar.java
@@ -25,6 +25,7 @@ import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.TimestampColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.serde2.io.DateWritable;
 import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector.PrimitiveCategory;
 import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo;
@@ -75,7 +76,7 @@ public class VectorUDFDateDiffColScalar extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     if (childExpressions != null) {
       super.evaluateChildren(batch);

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateDiffScalarCol.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateDiffScalarCol.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateDiffScalarCol.java
index c436c96..c51d3cd 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateDiffScalarCol.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateDiffScalarCol.java
@@ -24,6 +24,7 @@ import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.TimestampColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.serde2.io.DateWritable;
 import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector.PrimitiveCategory;
 import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo;
@@ -73,7 +74,7 @@ public class VectorUDFDateDiffScalarCol extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     if (childExpressions != null) {
       super.evaluateChildren(batch);

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFMapIndexBaseCol.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFMapIndexBaseCol.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFMapIndexBaseCol.java
index 1f2d5cb..157154a 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFMapIndexBaseCol.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFMapIndexBaseCol.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.hive.ql.exec.vector.expressions;
 import org.apache.hadoop.hive.ql.exec.vector.ColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.MapColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 /**
  * Superclass to support vectorized functions that take a column value as key of Map
@@ -45,7 +46,7 @@ public abstract class VectorUDFMapIndexBaseCol extends VectorUDFMapIndexBase {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
     if (childExpressions != null) {
       super.evaluateChildren(batch);
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFMapIndexBaseScalar.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFMapIndexBaseScalar.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFMapIndexBaseScalar.java
index a7d730b..72662e0 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFMapIndexBaseScalar.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFMapIndexBaseScalar.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.hive.ql.exec.vector.expressions;
 import org.apache.hadoop.hive.ql.exec.vector.ColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.MapColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 /**
  * Superclass to support vectorized functions that take a scalar as key of Map
@@ -42,7 +43,7 @@ public abstract class VectorUDFMapIndexBaseScalar extends VectorUDFMapIndexBase
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
     if (childExpressions != null) {
       super.evaluateChildren(batch);
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFStructField.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFStructField.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFStructField.java
index b40126a..342cb05 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFStructField.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFStructField.java
@@ -24,6 +24,7 @@ import org.apache.hadoop.hive.ql.exec.vector.ColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.StructColumnVector;
 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;
 
 /**
  * Vectorized instruction to get the field of Struct type with field name and put
@@ -45,7 +46,7 @@ public class VectorUDFStructField extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     // return immediately if batch is empty
     final int n = batch.size;

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFTimestampFieldDate.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFTimestampFieldDate.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFTimestampFieldDate.java
index eb6d6dd..a1167b2 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFTimestampFieldDate.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFTimestampFieldDate.java
@@ -74,7 +74,7 @@ public abstract class VectorUDFTimestampFieldDate extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     Preconditions.checkState(
         ((PrimitiveTypeInfo) inputTypeInfos[0]).getPrimitiveCategory() == PrimitiveCategory.DATE);

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFTimestampFieldString.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFTimestampFieldString.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFTimestampFieldString.java
index 2918546..931cc98 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFTimestampFieldString.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFTimestampFieldString.java
@@ -87,7 +87,7 @@ public abstract class VectorUDFTimestampFieldString extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     if (childExpressions != null) {
       super.evaluateChildren(batch);

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFTimestampFieldTimestamp.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFTimestampFieldTimestamp.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFTimestampFieldTimestamp.java
index 740a00c..a9ea1f6 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFTimestampFieldTimestamp.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFTimestampFieldTimestamp.java
@@ -74,7 +74,7 @@ public abstract class VectorUDFTimestampFieldTimestamp extends VectorExpression
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     Preconditions.checkState(
         ((PrimitiveTypeInfo) inputTypeInfos[0]).getPrimitiveCategory() == PrimitiveCategory.TIMESTAMP);

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinGenerateResultOperator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinGenerateResultOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinGenerateResultOperator.java
index 92ec1ee..3821cc6 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinGenerateResultOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinGenerateResultOperator.java
@@ -134,7 +134,7 @@ public abstract class VectorMapJoinGenerateResultOperator extends VectorMapJoinC
   //------------------------------------------------------------------------------------------------
 
   protected void performValueExpressions(VectorizedRowBatch batch,
-      int[] allMatchs, int allMatchCount) {
+      int[] allMatchs, int allMatchCount) throws HiveException {
     /*
      *  For the moment, pretend all matched are selected so we can evaluate the value
      *  expressions.

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinOuterGenerateResultOperator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinOuterGenerateResultOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinOuterGenerateResultOperator.java
index 8a6c817..2e5c568 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinOuterGenerateResultOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinOuterGenerateResultOperator.java
@@ -196,7 +196,7 @@ public abstract class VectorMapJoinOuterGenerateResultOperator
    *          The (original) input batch size.
    */
   private void doValueExprOnInputSelected(VectorizedRowBatch batch,
-      boolean inputSelectedInUse, int inputLogicalSize) {
+      boolean inputSelectedInUse, int inputLogicalSize) throws HiveException {
 
     int saveBatchSize = batch.size;
     int[] saveSelected = batch.selected;
@@ -228,7 +228,7 @@ public abstract class VectorMapJoinOuterGenerateResultOperator
    *          The size of selected.
    */
   private void doValueExpr(VectorizedRowBatch batch,
-      int[] selected, int size) {
+      int[] selected, int size) throws HiveException {
 
     int saveBatchSize = batch.size;
     int[] saveSelected = batch.selected;

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFEvaluatorBase.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFEvaluatorBase.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFEvaluatorBase.java
index 5957f15..ff09eb6 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFEvaluatorBase.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFEvaluatorBase.java
@@ -25,6 +25,7 @@ import org.apache.hadoop.hive.ql.exec.vector.ColumnVector.Type;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.IdentityExpression;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.plan.ptf.WindowFrameDef;
 import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
 
@@ -72,14 +73,14 @@ public abstract class VectorPTFEvaluatorBase {
   }
 
   // Evaluate the aggregation input argument expression.
-  public void evaluateInputExpr(VectorizedRowBatch batch) {
+  public void evaluateInputExpr(VectorizedRowBatch batch) throws HiveException {
     if (inputVecExpr != null) {
       inputVecExpr.evaluate(batch);
     }
   }
 
   // Evaluate the aggregation over one of the group's batches.
-  public abstract void evaluateGroupBatch(VectorizedRowBatch batch, boolean isLastGroupBatch);
+  public abstract void evaluateGroupBatch(VectorizedRowBatch batch, boolean isLastGroupBatch) throws HiveException;
 
   // Returns true if the aggregation result will be streamed.
   public boolean streamsResult() {

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFEvaluatorCount.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFEvaluatorCount.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFEvaluatorCount.java
index 9515832..f837148 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFEvaluatorCount.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFEvaluatorCount.java
@@ -25,6 +25,7 @@ import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.ColumnVector.Type;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.plan.ptf.WindowFrameDef;
 
 import com.google.common.base.Preconditions;
@@ -48,7 +49,9 @@ public class VectorPTFEvaluatorCount extends VectorPTFEvaluatorBase {
     resetEvaluator();
   }
 
-  public void evaluateGroupBatch(VectorizedRowBatch batch, boolean isLastGroupBatch) {
+  public void evaluateGroupBatch(VectorizedRowBatch batch, boolean isLastGroupBatch)
+      throws HiveException {
+
     evaluateInputExpr(batch);
 
     // Count non-null column rows.

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFEvaluatorDecimalAvg.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFEvaluatorDecimalAvg.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFEvaluatorDecimalAvg.java
index bd4896a..d0589d2 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFEvaluatorDecimalAvg.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFEvaluatorDecimalAvg.java
@@ -25,6 +25,7 @@ import org.apache.hadoop.hive.ql.exec.vector.DecimalColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.ColumnVector.Type;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.plan.ptf.WindowFrameDef;
 import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
 
@@ -56,7 +57,9 @@ public class VectorPTFEvaluatorDecimalAvg extends VectorPTFEvaluatorBase {
     resetEvaluator();
   }
 
-  public void evaluateGroupBatch(VectorizedRowBatch batch, boolean isLastGroupBatch) {
+  public void evaluateGroupBatch(VectorizedRowBatch batch, boolean isLastGroupBatch)
+      throws HiveException {
+
     evaluateInputExpr(batch);
 
     // Sum all non-null decimal column values for avg; maintain isGroupResultNull; after last row of

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFEvaluatorDecimalFirstValue.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFEvaluatorDecimalFirstValue.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFEvaluatorDecimalFirstValue.java
index daba90c..ce118bc 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFEvaluatorDecimalFirstValue.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFEvaluatorDecimalFirstValue.java
@@ -27,6 +27,7 @@ import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.ColumnVector.Type;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.plan.ptf.WindowFrameDef;
 import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
 
@@ -55,7 +56,9 @@ public class VectorPTFEvaluatorDecimalFirstValue extends VectorPTFEvaluatorBase
     resetEvaluator();
   }
 
-  public void evaluateGroupBatch(VectorizedRowBatch batch, boolean isLastGroupBatch) {
+  public void evaluateGroupBatch(VectorizedRowBatch batch, boolean isLastGroupBatch)
+      throws HiveException {
+
     evaluateInputExpr(batch);
 
     // First row determines isGroupResultNull and decimal firstValue; stream fill result as repeated.

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFEvaluatorDecimalLastValue.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFEvaluatorDecimalLastValue.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFEvaluatorDecimalLastValue.java
index b2cbdf6..7cc6158 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFEvaluatorDecimalLastValue.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFEvaluatorDecimalLastValue.java
@@ -26,6 +26,7 @@ import org.apache.hadoop.hive.ql.exec.vector.DecimalColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.ColumnVector.Type;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.plan.ptf.WindowFrameDef;
 import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
 
@@ -53,7 +54,9 @@ public class VectorPTFEvaluatorDecimalLastValue extends VectorPTFEvaluatorBase {
     resetEvaluator();
   }
 
-  public void evaluateGroupBatch(VectorizedRowBatch batch, boolean isLastGroupBatch) {
+  public void evaluateGroupBatch(VectorizedRowBatch batch, boolean isLastGroupBatch)
+      throws HiveException {
+
     evaluateInputExpr(batch);
 
     // Last row of last batch determines isGroupResultNull and decimal lastValue.

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFEvaluatorDecimalMax.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFEvaluatorDecimalMax.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFEvaluatorDecimalMax.java
index 4d8c7fe..ae4792e 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFEvaluatorDecimalMax.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFEvaluatorDecimalMax.java
@@ -26,6 +26,7 @@ import org.apache.hadoop.hive.ql.exec.vector.DecimalColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.ColumnVector.Type;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.plan.ptf.WindowFrameDef;
 import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
 
@@ -50,7 +51,9 @@ public class VectorPTFEvaluatorDecimalMax extends VectorPTFEvaluatorBase {
     resetEvaluator();
   }
 
-  public void evaluateGroupBatch(VectorizedRowBatch batch, boolean isLastGroupBatch) {
+  public void evaluateGroupBatch(VectorizedRowBatch batch, boolean isLastGroupBatch)
+      throws HiveException {
+
     evaluateInputExpr(batch);
 
     // Determine maximum of all non-null decimal column values; maintain isGroupResultNull.

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFEvaluatorDecimalMin.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFEvaluatorDecimalMin.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFEvaluatorDecimalMin.java
index 312c43c..521a2ff 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFEvaluatorDecimalMin.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFEvaluatorDecimalMin.java
@@ -26,6 +26,7 @@ import org.apache.hadoop.hive.ql.exec.vector.DecimalColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.ColumnVector.Type;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.plan.ptf.WindowFrameDef;
 import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
 
@@ -50,7 +51,9 @@ public class VectorPTFEvaluatorDecimalMin extends VectorPTFEvaluatorBase {
     resetEvaluator();
   }
 
-  public void evaluateGroupBatch(VectorizedRowBatch batch, boolean isLastGroupBatch) {
+  public void evaluateGroupBatch(VectorizedRowBatch batch, boolean isLastGroupBatch)
+      throws HiveException {
+
     evaluateInputExpr(batch);
 
     // Determine minimum of all non-null decimal column values; maintain isGroupResultNull.

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFEvaluatorDecimalSum.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFEvaluatorDecimalSum.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFEvaluatorDecimalSum.java
index e899c36..5a7dc4c 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFEvaluatorDecimalSum.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFEvaluatorDecimalSum.java
@@ -25,6 +25,7 @@ import org.apache.hadoop.hive.ql.exec.vector.DecimalColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.ColumnVector.Type;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.plan.ptf.WindowFrameDef;
 import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
 
@@ -51,7 +52,9 @@ public class VectorPTFEvaluatorDecimalSum extends VectorPTFEvaluatorBase {
     resetEvaluator();
   }
 
-  public void evaluateGroupBatch(VectorizedRowBatch batch, boolean isLastGroupBatch) {
+  public void evaluateGroupBatch(VectorizedRowBatch batch, boolean isLastGroupBatch)
+      throws HiveException {
+
     evaluateInputExpr(batch);
 
     // Sum all non-null decimal column values; maintain isGroupResultNull.

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFEvaluatorDenseRank.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFEvaluatorDenseRank.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFEvaluatorDenseRank.java
index e457e32..f639316 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFEvaluatorDenseRank.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFEvaluatorDenseRank.java
@@ -24,6 +24,7 @@ import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.ColumnVector.Type;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.plan.ptf.WindowFrameDef;
 
 /**
@@ -46,7 +47,9 @@ public class VectorPTFEvaluatorDenseRank extends VectorPTFEvaluatorBase {
     resetEvaluator();
   }
 
-  public void evaluateGroupBatch(VectorizedRowBatch batch, boolean isLastGroupBatch) {
+  public void evaluateGroupBatch(VectorizedRowBatch batch, boolean isLastGroupBatch)
+      throws HiveException {
+
     evaluateInputExpr(batch);
 
     LongColumnVector longColVector = (LongColumnVector) batch.cols[outputColumnNum];

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFEvaluatorDoubleAvg.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFEvaluatorDoubleAvg.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFEvaluatorDoubleAvg.java
index 298de2d..cd09ce7 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFEvaluatorDoubleAvg.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFEvaluatorDoubleAvg.java
@@ -24,6 +24,7 @@ import org.apache.hadoop.hive.ql.exec.vector.DoubleColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.ColumnVector.Type;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.plan.ptf.WindowFrameDef;
 
 import com.google.common.base.Preconditions;
@@ -50,7 +51,9 @@ public class VectorPTFEvaluatorDoubleAvg extends VectorPTFEvaluatorBase {
     resetEvaluator();
   }
 
-  public void evaluateGroupBatch(VectorizedRowBatch batch, boolean isLastGroupBatch) {
+  public void evaluateGroupBatch(VectorizedRowBatch batch, boolean isLastGroupBatch)
+      throws HiveException {
+
     evaluateInputExpr(batch);
 
     // Sum all non-null double column values for avg; maintain isGroupResultNull; after last row of

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFEvaluatorDoubleFirstValue.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFEvaluatorDoubleFirstValue.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFEvaluatorDoubleFirstValue.java
index 460fbe5..7d1fb61 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFEvaluatorDoubleFirstValue.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFEvaluatorDoubleFirstValue.java
@@ -25,6 +25,7 @@ import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.ColumnVector.Type;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.plan.ptf.WindowFrameDef;
 
 import com.google.common.base.Preconditions;
@@ -51,7 +52,9 @@ public class VectorPTFEvaluatorDoubleFirstValue extends VectorPTFEvaluatorBase {
     resetEvaluator();
   }
 
-  public void evaluateGroupBatch(VectorizedRowBatch batch, boolean isLastGroupBatch) {
+  public void evaluateGroupBatch(VectorizedRowBatch batch, boolean isLastGroupBatch)
+      throws HiveException {
+
     evaluateInputExpr(batch);
 
     // First row determines isGroupResultNull and double firstValue; stream fill result as repeated.

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFEvaluatorDoubleLastValue.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFEvaluatorDoubleLastValue.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFEvaluatorDoubleLastValue.java
index 51d06a8..dbc78eb 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFEvaluatorDoubleLastValue.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFEvaluatorDoubleLastValue.java
@@ -24,6 +24,7 @@ import org.apache.hadoop.hive.ql.exec.vector.DoubleColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.ColumnVector.Type;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.plan.ptf.WindowFrameDef;
 
 import com.google.common.base.Preconditions;
@@ -49,7 +50,9 @@ public class VectorPTFEvaluatorDoubleLastValue extends VectorPTFEvaluatorBase {
     resetEvaluator();
   }
 
-  public void evaluateGroupBatch(VectorizedRowBatch batch, boolean isLastGroupBatch) {
+  public void evaluateGroupBatch(VectorizedRowBatch batch, boolean isLastGroupBatch)
+      throws HiveException {
+
     evaluateInputExpr(batch);
 
     // Last row of last batch determines isGroupResultNull and double lastValue.

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFEvaluatorDoubleMax.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFEvaluatorDoubleMax.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFEvaluatorDoubleMax.java
index 2bc4c6b..93745c4 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFEvaluatorDoubleMax.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFEvaluatorDoubleMax.java
@@ -24,6 +24,7 @@ import org.apache.hadoop.hive.ql.exec.vector.DoubleColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.ColumnVector.Type;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.plan.ptf.WindowFrameDef;
 
 import com.google.common.base.Preconditions;
@@ -46,7 +47,9 @@ public class VectorPTFEvaluatorDoubleMax extends VectorPTFEvaluatorBase {
     resetEvaluator();
   }
 
-  public void evaluateGroupBatch(VectorizedRowBatch batch, boolean isLastGroupBatch) {
+  public void evaluateGroupBatch(VectorizedRowBatch batch, boolean isLastGroupBatch)
+      throws HiveException {
+
     evaluateInputExpr(batch);
 
     // Determine maximum of all non-null double column values; maintain isGroupResultNull.

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFEvaluatorDoubleMin.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFEvaluatorDoubleMin.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFEvaluatorDoubleMin.java
index 6a422b2..42cce54 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFEvaluatorDoubleMin.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFEvaluatorDoubleMin.java
@@ -24,6 +24,7 @@ import org.apache.hadoop.hive.ql.exec.vector.DoubleColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.ColumnVector.Type;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.plan.ptf.WindowFrameDef;
 
 import com.google.common.base.Preconditions;
@@ -46,7 +47,9 @@ public class VectorPTFEvaluatorDoubleMin extends VectorPTFEvaluatorBase {
     resetEvaluator();
   }
 
-  public void evaluateGroupBatch(VectorizedRowBatch batch, boolean isLastGroupBatch) {
+  public void evaluateGroupBatch(VectorizedRowBatch batch, boolean isLastGroupBatch)
+      throws HiveException {
+
     evaluateInputExpr(batch);
 
     // Determine minimum of all non-null double column values; maintain isGroupResultNull.

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFEvaluatorDoubleSum.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFEvaluatorDoubleSum.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFEvaluatorDoubleSum.java
index 2ecc4ae..552d4a9 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFEvaluatorDoubleSum.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFEvaluatorDoubleSum.java
@@ -24,6 +24,7 @@ import org.apache.hadoop.hive.ql.exec.vector.DoubleColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.ColumnVector.Type;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.plan.ptf.WindowFrameDef;
 
 import com.google.common.base.Preconditions;
@@ -46,7 +47,9 @@ public class VectorPTFEvaluatorDoubleSum extends VectorPTFEvaluatorBase {
     resetEvaluator();
   }
 
-  public void evaluateGroupBatch(VectorizedRowBatch batch, boolean isLastGroupBatch) {
+  public void evaluateGroupBatch(VectorizedRowBatch batch, boolean isLastGroupBatch)
+      throws HiveException {
+
     evaluateInputExpr(batch);
 
     // Sum all non-null double column values; maintain isGroupResultNull.

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFEvaluatorLongAvg.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFEvaluatorLongAvg.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFEvaluatorLongAvg.java
index f48df25..ad3950f 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFEvaluatorLongAvg.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFEvaluatorLongAvg.java
@@ -24,6 +24,7 @@ import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.ColumnVector.Type;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.plan.ptf.WindowFrameDef;
 
 import com.google.common.base.Preconditions;
@@ -50,7 +51,9 @@ public class VectorPTFEvaluatorLongAvg extends VectorPTFEvaluatorBase {
     resetEvaluator();
   }
 
-  public void evaluateGroupBatch(VectorizedRowBatch batch, boolean isLastGroupBatch) {
+  public void evaluateGroupBatch(VectorizedRowBatch batch, boolean isLastGroupBatch)
+      throws HiveException {
+
     evaluateInputExpr(batch);
 
     // Sum all non-null long column values for avg; maintain isGroupResultNull; after last row of

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFEvaluatorLongFirstValue.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFEvaluatorLongFirstValue.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFEvaluatorLongFirstValue.java
index 3deadb1..a288f16 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFEvaluatorLongFirstValue.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFEvaluatorLongFirstValue.java
@@ -25,6 +25,7 @@ import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.ColumnVector.Type;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.plan.ptf.WindowFrameDef;
 
 import com.google.common.base.Preconditions;
@@ -51,7 +52,9 @@ public class VectorPTFEvaluatorLongFirstValue extends VectorPTFEvaluatorBase {
     resetEvaluator();
   }
 
-  public void evaluateGroupBatch(VectorizedRowBatch batch, boolean isLastGroupBatch) {
+  public void evaluateGroupBatch(VectorizedRowBatch batch, boolean isLastGroupBatch)
+      throws HiveException {
+
     evaluateInputExpr(batch);
 
     // First row determines isGroupResultNull and long firstValue; stream fill result as repeated.

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFEvaluatorLongLastValue.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFEvaluatorLongLastValue.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFEvaluatorLongLastValue.java
index 0a5df51..cf7bf24 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFEvaluatorLongLastValue.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFEvaluatorLongLastValue.java
@@ -24,6 +24,7 @@ import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.ColumnVector.Type;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.plan.ptf.WindowFrameDef;
 
 import com.google.common.base.Preconditions;
@@ -49,7 +50,9 @@ public class VectorPTFEvaluatorLongLastValue extends VectorPTFEvaluatorBase {
     resetEvaluator();
   }
 
-  public void evaluateGroupBatch(VectorizedRowBatch batch, boolean isLastGroupBatch) {
+  public void evaluateGroupBatch(VectorizedRowBatch batch, boolean isLastGroupBatch)
+      throws HiveException {
+
     evaluateInputExpr(batch);
 
     // Last row of last batch determines isGroupResultNull and long lastValue.

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFEvaluatorLongMax.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFEvaluatorLongMax.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFEvaluatorLongMax.java
index 0e7eb07..04f106b 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFEvaluatorLongMax.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFEvaluatorLongMax.java
@@ -24,6 +24,7 @@ import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.ColumnVector.Type;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.plan.ptf.WindowFrameDef;
 
 import com.google.common.base.Preconditions;
@@ -46,7 +47,9 @@ public class VectorPTFEvaluatorLongMax extends VectorPTFEvaluatorBase {
     resetEvaluator();
   }
 
-  public void evaluateGroupBatch(VectorizedRowBatch batch, boolean isLastGroupBatch) {
+  public void evaluateGroupBatch(VectorizedRowBatch batch, boolean isLastGroupBatch)
+      throws HiveException {
+
     evaluateInputExpr(batch);
 
     // Determine maximum of all non-null long column values; maintain isGroupResultNull.

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFEvaluatorLongMin.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFEvaluatorLongMin.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFEvaluatorLongMin.java
index df31c51..a80819d 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFEvaluatorLongMin.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFEvaluatorLongMin.java
@@ -24,6 +24,7 @@ import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.ColumnVector.Type;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.plan.ptf.WindowFrameDef;
 
 import com.google.common.base.Preconditions;
@@ -46,7 +47,9 @@ public class VectorPTFEvaluatorLongMin extends VectorPTFEvaluatorBase {
     resetEvaluator();
   }
 
-  public void evaluateGroupBatch(VectorizedRowBatch batch, boolean isLastGroupBatch) {
+  public void evaluateGroupBatch(VectorizedRowBatch batch, boolean isLastGroupBatch)
+      throws HiveException {
+
     evaluateInputExpr(batch);
 
     // Determine minimum of all non-null long column values; maintain isGroupResultNull.

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFEvaluatorLongSum.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFEvaluatorLongSum.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFEvaluatorLongSum.java
index 9402218..277cff3 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFEvaluatorLongSum.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFEvaluatorLongSum.java
@@ -24,6 +24,7 @@ import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.ColumnVector.Type;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.plan.ptf.WindowFrameDef;
 
 import com.google.common.base.Preconditions;
@@ -46,7 +47,9 @@ public class VectorPTFEvaluatorLongSum extends VectorPTFEvaluatorBase {
     resetEvaluator();
   }
 
-  public void evaluateGroupBatch(VectorizedRowBatch batch, boolean isLastGroupBatch) {
+  public void evaluateGroupBatch(VectorizedRowBatch batch, boolean isLastGroupBatch)
+      throws HiveException {
+
     evaluateInputExpr(batch);
 
     // Sum all non-null long column values; maintain isGroupResultNull.

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFEvaluatorRank.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFEvaluatorRank.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFEvaluatorRank.java
index 34add61..9b2710f 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFEvaluatorRank.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFEvaluatorRank.java
@@ -24,6 +24,7 @@ import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.ColumnVector.Type;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.plan.ptf.WindowFrameDef;
 
 /**
@@ -47,7 +48,9 @@ public class VectorPTFEvaluatorRank extends VectorPTFEvaluatorBase {
     resetEvaluator();
   }
 
-  public void evaluateGroupBatch(VectorizedRowBatch batch, boolean isLastGroupBatch) {
+  public void evaluateGroupBatch(VectorizedRowBatch batch, boolean isLastGroupBatch)
+      throws HiveException {
+
     evaluateInputExpr(batch);
 
     /*

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFEvaluatorRowNumber.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFEvaluatorRowNumber.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFEvaluatorRowNumber.java
index 8d43b33..3bc70c8 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFEvaluatorRowNumber.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFEvaluatorRowNumber.java
@@ -24,6 +24,7 @@ import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.ColumnVector.Type;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.plan.ptf.WindowFrameDef;
 
 /**
@@ -45,7 +46,9 @@ public class VectorPTFEvaluatorRowNumber extends VectorPTFEvaluatorBase {
     resetEvaluator();
   }
 
-  public void evaluateGroupBatch(VectorizedRowBatch batch, boolean isLastGroupBatch) {
+  public void evaluateGroupBatch(VectorizedRowBatch batch, boolean isLastGroupBatch)
+      throws HiveException {
+
     evaluateInputExpr(batch);
 
     final int size = batch.size;

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFGroupBatches.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFGroupBatches.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFGroupBatches.java
index 607d34a..573910e 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFGroupBatches.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFGroupBatches.java
@@ -163,7 +163,8 @@ public class VectorPTFGroupBatches {
     spillRowBytesContainer = null;
   }
 
-  public void evaluateStreamingGroupBatch(VectorizedRowBatch batch, boolean isLastGroupBatch) {
+  public void evaluateStreamingGroupBatch(VectorizedRowBatch batch, boolean isLastGroupBatch)
+      throws HiveException {
 
     // Streaming evaluators fill in their results during the evaluate call.
     for (VectorPTFEvaluatorBase evaluator : evaluators) {
@@ -171,7 +172,9 @@ public class VectorPTFGroupBatches {
     }
   }
 
-  public void evaluateGroupBatch(VectorizedRowBatch batch, boolean isLastGroupBatch) {
+  public void evaluateGroupBatch(VectorizedRowBatch batch, boolean isLastGroupBatch)
+      throws HiveException {
+
     for (VectorPTFEvaluatorBase evaluator : evaluators) {
       evaluator.evaluateGroupBatch(batch, isLastGroupBatch);
     }


[21/50] [abbrv] hive git commit: HIVE-19453 : Extend Load Data statement to take Input file format and Serde as parameters (Deepak Jaiswal, reviewed by Jason Dere)

Posted by vg...@apache.org.
HIVE-19453 : Extend Load Data statement to take Input file format and Serde as parameters (Deepak Jaiswal, reviewed by Jason Dere)


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

Branch: refs/heads/branch-3.0.0
Commit: 32e29cc63c41d722a4b2f8ffae4b9c3a660b8db4
Parents: b331338
Author: Deepak Jaiswal <dj...@apache.org>
Authored: Wed May 9 11:06:34 2018 -0700
Committer: Deepak Jaiswal <dj...@apache.org>
Committed: Fri May 11 10:55:14 2018 -0700

----------------------------------------------------------------------
 .../apache/hadoop/hive/ql/parse/HiveParser.g    | 12 +++++--
 .../hive/ql/parse/LoadSemanticAnalyzer.java     | 33 ++++++++++++++++++--
 .../clientpositive/load_data_using_job.q        |  8 +++--
 .../llap/load_data_using_job.q.out              |  8 +++++
 4 files changed, 54 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/32e29cc6/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g b/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g
index a837d67..3712a53 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g
@@ -422,6 +422,7 @@ TOK_ADD_TRIGGER;
 TOK_REPLACE;
 TOK_LIKERP;
 TOK_UNMANAGED;
+TOK_INPUTFORMAT;
 }
 
 
@@ -835,8 +836,8 @@ execStatement
 loadStatement
 @init { pushMsg("load statement", state); }
 @after { popMsg(state); }
-    : KW_LOAD KW_DATA (islocal=KW_LOCAL)? KW_INPATH (path=StringLiteral) (isoverwrite=KW_OVERWRITE)? KW_INTO KW_TABLE (tab=tableOrPartition)
-    -> ^(TOK_LOAD $path $tab $islocal? $isoverwrite?)
+    : KW_LOAD KW_DATA (islocal=KW_LOCAL)? KW_INPATH (path=StringLiteral) (isoverwrite=KW_OVERWRITE)? KW_INTO KW_TABLE (tab=tableOrPartition) inputFileFormat?
+    -> ^(TOK_LOAD $path $tab $islocal? $isoverwrite? inputFileFormat?)
     ;
 
 replicationClause
@@ -1489,6 +1490,13 @@ fileFormat
     | genericSpec=identifier -> ^(TOK_FILEFORMAT_GENERIC $genericSpec)
     ;
 
+inputFileFormat
+@init { pushMsg("Load Data input file format specification", state); }
+@after { popMsg(state); }
+    : KW_INPUTFORMAT inFmt=StringLiteral KW_SERDE serdeCls=StringLiteral
+      -> ^(TOK_INPUTFORMAT $inFmt $serdeCls)
+    ;
+
 tabTypeExpr
 @init { pushMsg("specifying table types", state); }
 @after { popMsg(state); }

http://git-wip-us.apache.org/repos/asf/hive/blob/32e29cc6/ql/src/java/org/apache/hadoop/hive/ql/parse/LoadSemanticAnalyzer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/LoadSemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/LoadSemanticAnalyzer.java
index 2b88ea6..866f43d 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/LoadSemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/LoadSemanticAnalyzer.java
@@ -79,6 +79,8 @@ public class LoadSemanticAnalyzer extends SemanticAnalyzer {
   // AST specific data
   private Tree fromTree, tableTree;
   private boolean isLocal = false, isOverWrite = false;
+  private String inputFormatClassName = null;
+  private String serDeClassName = null;
 
   public LoadSemanticAnalyzer(QueryState queryState) throws SemanticException {
     super(queryState);
@@ -257,12 +259,30 @@ public class LoadSemanticAnalyzer extends SemanticAnalyzer {
     fromTree = ast.getChild(0);
     tableTree = ast.getChild(1);
 
-    if (ast.getChildCount() == 4) {
+    boolean inputInfo = false;
+    // Check the last node
+    ASTNode child = (ASTNode)ast.getChild(ast.getChildCount() - 1);
+    if (child.getToken().getType() == HiveParser.TOK_INPUTFORMAT) {
+      if (child.getChildCount() != 2) {
+        throw new SemanticException("FileFormat should contain both input format and Serde");
+      }
+      try {
+        inputFormatClassName = stripQuotes(child.getChild(0).getText());
+        serDeClassName = stripQuotes(child.getChild(1).getText());
+        inputInfo = true;
+      } catch (Exception e) {
+        throw new SemanticException("FileFormat inputFormatClassName or serDeClassName is incorrect");
+      }
+    }
+
+    if ((!inputInfo && ast.getChildCount() == 4) ||
+        (inputInfo && ast.getChildCount() == 5)) {
       isLocal = true;
       isOverWrite = true;
     }
 
-    if (ast.getChildCount() == 3) {
+    if ((!inputInfo && ast.getChildCount() == 3) ||
+        (inputInfo && ast.getChildCount() == 4)) {
       if (ast.getChild(2).getText().toLowerCase().equals("local")) {
         isLocal = true;
       } else {
@@ -450,7 +470,14 @@ public class LoadSemanticAnalyzer extends SemanticAnalyzer {
 
     // Set data location and input format, it must be text
     tempTableObj.setDataLocation(new Path(fromURI));
-    tempTableObj.setInputFormatClass(TextInputFormat.class);
+    if (inputFormatClassName != null && serDeClassName != null) {
+      try {
+        tempTableObj.setInputFormatClass(inputFormatClassName);
+        tempTableObj.setSerializationLib(serDeClassName);
+      } catch (HiveException e) {
+        throw new SemanticException("Load Data: Failed to set inputFormat or SerDe");
+      }
+    }
 
     // Step 2 : create the Insert query
     StringBuilder rewrittenQueryStr = new StringBuilder();

http://git-wip-us.apache.org/repos/asf/hive/blob/32e29cc6/ql/src/test/queries/clientpositive/load_data_using_job.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/load_data_using_job.q b/ql/src/test/queries/clientpositive/load_data_using_job.q
index 3928f1f..3659b6e 100644
--- a/ql/src/test/queries/clientpositive/load_data_using_job.q
+++ b/ql/src/test/queries/clientpositive/load_data_using_job.q
@@ -84,7 +84,11 @@ drop table srcbucket_mapjoin;
 
 -- Load into ORC table using text files
 CREATE TABLE srcbucket_mapjoin(key int, value string) partitioned by (ds string) STORED AS ORC;
-explain load data local inpath '../../data/files/load_data_job/load_data_1_partition.txt' INTO TABLE srcbucket_mapjoin;
-load data local inpath '../../data/files/load_data_job/load_data_1_partition.txt' INTO TABLE srcbucket_mapjoin;
+explain load data local inpath '../../data/files/load_data_job/load_data_1_partition.txt' INTO TABLE srcbucket_mapjoin
+INPUTFORMAT 'org.apache.hadoop.mapred.TextInputFormat'
+SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe';
+load data local inpath '../../data/files/load_data_job/load_data_1_partition.txt' INTO TABLE srcbucket_mapjoin
+INPUTFORMAT 'org.apache.hadoop.mapred.TextInputFormat'
+SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe';
 select * from srcbucket_mapjoin;
 drop table srcbucket_mapjoin;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/32e29cc6/ql/src/test/results/clientpositive/llap/load_data_using_job.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/load_data_using_job.q.out b/ql/src/test/results/clientpositive/llap/load_data_using_job.q.out
index 116630c..c3b70a3 100644
--- a/ql/src/test/results/clientpositive/llap/load_data_using_job.q.out
+++ b/ql/src/test/results/clientpositive/llap/load_data_using_job.q.out
@@ -2776,8 +2776,12 @@ POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@srcbucket_mapjoin
 PREHOOK: query: explain load data local inpath '../../data/files/load_data_job/load_data_1_partition.txt' INTO TABLE srcbucket_mapjoin
+INPUTFORMAT 'org.apache.hadoop.mapred.TextInputFormat'
+SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe'
 PREHOOK: type: QUERY
 POSTHOOK: query: explain load data local inpath '../../data/files/load_data_job/load_data_1_partition.txt' INTO TABLE srcbucket_mapjoin
+INPUTFORMAT 'org.apache.hadoop.mapred.TextInputFormat'
+SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe'
 POSTHOOK: type: QUERY
 STAGE DEPENDENCIES:
   Stage-1 is a root stage
@@ -2830,10 +2834,14 @@ STAGE PLANS:
       Basic Stats Work:
 
 PREHOOK: query: load data local inpath '../../data/files/load_data_job/load_data_1_partition.txt' INTO TABLE srcbucket_mapjoin
+INPUTFORMAT 'org.apache.hadoop.mapred.TextInputFormat'
+SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe'
 PREHOOK: type: QUERY
 PREHOOK: Input: default@srcbucket_mapjoin__TEMP_TABLE_FOR_LOAD_DATA__
 PREHOOK: Output: default@srcbucket_mapjoin
 POSTHOOK: query: load data local inpath '../../data/files/load_data_job/load_data_1_partition.txt' INTO TABLE srcbucket_mapjoin
+INPUTFORMAT 'org.apache.hadoop.mapred.TextInputFormat'
+SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe'
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@srcbucket_mapjoin__TEMP_TABLE_FOR_LOAD_DATA__
 POSTHOOK: Output: default@srcbucket_mapjoin@ds=2008-04-08


[42/50] [abbrv] hive git commit: HIVE-19497: SessionHiveMetaStoreClient.getTable should respect default catalog (Dongjoon Hyun, reviewed by Alan Gates)

Posted by vg...@apache.org.
HIVE-19497: SessionHiveMetaStoreClient.getTable should respect default catalog (Dongjoon Hyun, reviewed by Alan Gates)


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

Branch: refs/heads/branch-3.0.0
Commit: c740e32fcf46f7eba5200419baa68f624cfa3abe
Parents: 3e04cc1
Author: Dongjoon Hyun <do...@apache.org>
Authored: Mon May 14 15:17:15 2018 -0700
Committer: Vineet Garg <vg...@apache.org>
Committed: Mon May 14 15:19:05 2018 -0700

----------------------------------------------------------------------
 .../apache/hadoop/hive/ql/metadata/SessionHiveMetaStoreClient.java | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/c740e32f/ql/src/java/org/apache/hadoop/hive/ql/metadata/SessionHiveMetaStoreClient.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/SessionHiveMetaStoreClient.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/SessionHiveMetaStoreClient.java
index 071756d..0d2ed54 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/SessionHiveMetaStoreClient.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/SessionHiveMetaStoreClient.java
@@ -166,7 +166,7 @@ public class SessionHiveMetaStoreClient extends HiveMetaStoreClient implements I
       return deepCopy(table);  // Original method used deepCopy(), do the same here.
     }
     // Try underlying client
-    return super.getTable(DEFAULT_CATALOG_NAME, dbname, name);
+    return super.getTable(MetaStoreUtils.getDefaultCatalog(conf), dbname, name);
   }
 
   // Need to override this one too or dropTable breaks because it doesn't find the table when checks


[26/50] [abbrv] hive git commit: HIVE-19517: Disable/delete TestNegativeCliDriver merge_negative_5 and mm_concatenate(Vineet Garg, reviewed by Vihang Karajgaonkar)

Posted by vg...@apache.org.
HIVE-19517: Disable/delete TestNegativeCliDriver merge_negative_5 and mm_concatenate(Vineet Garg, reviewed by Vihang Karajgaonkar)


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

Branch: refs/heads/branch-3.0.0
Commit: 9ebb2ff67a59337ab9258a124d090241ddf004b3
Parents: 2e5f863
Author: Vineet Garg <vg...@apache.org>
Authored: Sun May 13 16:43:09 2018 -0700
Committer: Vineet Garg <vg...@apache.org>
Committed: Sun May 13 16:49:23 2018 -0700

----------------------------------------------------------------------
 .../main/java/org/apache/hadoop/hive/cli/control/CliConfigs.java   | 2 ++
 1 file changed, 2 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/9ebb2ff6/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CliConfigs.java
----------------------------------------------------------------------
diff --git a/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CliConfigs.java b/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CliConfigs.java
index 1e65569..92bace2 100644
--- a/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CliConfigs.java
+++ b/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CliConfigs.java
@@ -360,6 +360,8 @@ public class CliConfigs {
         excludeQuery("authorization_uri_import.q");
         excludeQuery("spark_job_max_tasks.q");
         excludeQuery("spark_stage_max_tasks.q");
+        excludeQuery("merge_negative_5.q");
+        excludeQuery("mm_concatenate.q");
 
         setResultsDir("ql/src/test/results/clientnegative");
         setLogDir("itests/qtest/target/qfile-results/clientnegative");


[15/50] [abbrv] hive git commit: HIVE-19471 : bucket_map_join_tez1 and bucket_map_join_tez2 are failing (Deepak Jaiswal, reviewed by Vineet Garg)

Posted by vg...@apache.org.
HIVE-19471 : bucket_map_join_tez1 and bucket_map_join_tez2 are failing (Deepak Jaiswal, reviewed by Vineet Garg)


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

Branch: refs/heads/branch-3.0.0
Commit: 3c3880c08acdfd2c2e13e3055b7cc29791ffeb22
Parents: 5bb3df0
Author: Deepak Jaiswal <dj...@apache.org>
Authored: Thu May 10 11:28:46 2018 -0700
Committer: Deepak Jaiswal <dj...@apache.org>
Committed: Thu May 10 11:29:38 2018 -0700

----------------------------------------------------------------------
 .../llap/bucket_map_join_tez1.q.out             | 309 ++++++++++---------
 .../llap/bucket_map_join_tez2.q.out             | 156 +++++-----
 2 files changed, 226 insertions(+), 239 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/3c3880c0/ql/src/test/results/clientpositive/llap/bucket_map_join_tez1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/bucket_map_join_tez1.q.out b/ql/src/test/results/clientpositive/llap/bucket_map_join_tez1.q.out
index 116e00b..c1c342b 100644
--- a/ql/src/test/results/clientpositive/llap/bucket_map_join_tez1.q.out
+++ b/ql/src/test/results/clientpositive/llap/bucket_map_join_tez1.q.out
@@ -235,25 +235,25 @@ STAGE PLANS:
                   0 _col0 (type: int)
                   1 _col0 (type: int)
                 outputColumnNames: _col0, _col1, _col3
-                Statistics: Num rows: 400 Data size: 74400 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 391 Data size: 72726 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col0 (type: int), _col1 (type: string), _col3 (type: string)
                   outputColumnNames: _col0, _col1, _col2
-                  Statistics: Num rows: 400 Data size: 74400 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 391 Data size: 72726 Basic stats: COMPLETE Column stats: COMPLETE
                   Reduce Output Operator
                     key expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string)
                     sort order: +++
-                    Statistics: Num rows: 400 Data size: 74400 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 391 Data size: 72726 Basic stats: COMPLETE Column stats: COMPLETE
         Reducer 3 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Select Operator
                 expressions: KEY.reducesinkkey0 (type: int), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey2 (type: string)
                 outputColumnNames: _col0, _col1, _col2
-                Statistics: Num rows: 400 Data size: 74400 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 391 Data size: 72726 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 400 Data size: 74400 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 391 Data size: 72726 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -823,15 +823,15 @@ STAGE PLANS:
                         outputColumnNames: _col0, _col1, _col3
                         input vertices:
                           0 Map 1
-                        Statistics: Num rows: 400 Data size: 74400 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 391 Data size: 72726 Basic stats: COMPLETE Column stats: COMPLETE
                         Select Operator
                           expressions: _col0 (type: int), _col1 (type: string), _col3 (type: string)
                           outputColumnNames: _col0, _col1, _col2
-                          Statistics: Num rows: 400 Data size: 74400 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 391 Data size: 72726 Basic stats: COMPLETE Column stats: COMPLETE
                           Reduce Output Operator
                             key expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string)
                             sort order: +++
-                            Statistics: Num rows: 400 Data size: 74400 Basic stats: COMPLETE Column stats: COMPLETE
+                            Statistics: Num rows: 391 Data size: 72726 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Reducer 3 
@@ -840,10 +840,10 @@ STAGE PLANS:
               Select Operator
                 expressions: KEY.reducesinkkey0 (type: int), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey2 (type: string)
                 outputColumnNames: _col0, _col1, _col2
-                Statistics: Num rows: 400 Data size: 74400 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 391 Data size: 72726 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 400 Data size: 74400 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 391 Data size: 72726 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -1436,7 +1436,7 @@ STAGE PLANS:
                 keys:
                   0 _col0 (type: int)
                   1 _col0 (type: int)
-                Statistics: Num rows: 200 Data size: 1600 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 195 Data size: 1560 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: count()
                   mode: hash
@@ -1565,7 +1565,7 @@ STAGE PLANS:
                     1 _col0 (type: int)
                   input vertices:
                     1 Map 4
-                  Statistics: Num rows: 200 Data size: 1600 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 195 Data size: 1560 Basic stats: COMPLETE Column stats: COMPLETE
                   Group By Operator
                     aggregations: count()
                     mode: hash
@@ -1696,7 +1696,7 @@ STAGE PLANS:
                 keys:
                   0 _col0 (type: int)
                   1 _col0 (type: int)
-                Statistics: Num rows: 662 Data size: 5296 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 632 Data size: 5056 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: count()
                   mode: hash
@@ -1731,16 +1731,16 @@ STAGE PLANS:
                   0 _col0 (type: int)
                   1 _col0 (type: int)
                 outputColumnNames: _col1
-                Statistics: Num rows: 400 Data size: 1600 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 391 Data size: 1564 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col1 (type: int)
                   outputColumnNames: _col0
-                  Statistics: Num rows: 400 Data size: 1600 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 391 Data size: 1564 Basic stats: COMPLETE Column stats: COMPLETE
                   Reduce Output Operator
                     key expressions: _col0 (type: int)
                     sort order: +
                     Map-reduce partition columns: _col0 (type: int)
-                    Statistics: Num rows: 400 Data size: 1600 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 391 Data size: 1564 Basic stats: COMPLETE Column stats: COMPLETE
 
   Stage: Stage-0
     Fetch Operator
@@ -1838,11 +1838,11 @@ STAGE PLANS:
                         outputColumnNames: _col1
                         input vertices:
                           1 Map 4
-                        Statistics: Num rows: 400 Data size: 1600 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 391 Data size: 1564 Basic stats: COMPLETE Column stats: COMPLETE
                         Select Operator
                           expressions: _col1 (type: int)
                           outputColumnNames: _col0
-                          Statistics: Num rows: 400 Data size: 1600 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 391 Data size: 1564 Basic stats: COMPLETE Column stats: COMPLETE
                           Map Join Operator
                             condition map:
                                  Inner Join 0 to 1
@@ -1851,7 +1851,7 @@ STAGE PLANS:
                               1 _col0 (type: int)
                             input vertices:
                               0 Map 1
-                            Statistics: Num rows: 662 Data size: 5296 Basic stats: COMPLETE Column stats: COMPLETE
+                            Statistics: Num rows: 632 Data size: 5056 Basic stats: COMPLETE Column stats: COMPLETE
                             Group By Operator
                               aggregations: count()
                               mode: hash
@@ -2007,7 +2007,7 @@ STAGE PLANS:
                 keys:
                   0 _col0 (type: int)
                   1 _col0 (type: int)
-                Statistics: Num rows: 662 Data size: 5296 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 632 Data size: 5056 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: count()
                   mode: hash
@@ -2042,16 +2042,16 @@ STAGE PLANS:
                   0 _col0 (type: int)
                   1 _col0 (type: int)
                 outputColumnNames: _col1
-                Statistics: Num rows: 400 Data size: 1600 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 391 Data size: 1564 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col1 (type: int)
                   outputColumnNames: _col0
-                  Statistics: Num rows: 400 Data size: 1600 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 391 Data size: 1564 Basic stats: COMPLETE Column stats: COMPLETE
                   Reduce Output Operator
                     key expressions: _col0 (type: int)
                     sort order: +
                     Map-reduce partition columns: _col0 (type: int)
-                    Statistics: Num rows: 400 Data size: 1600 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 391 Data size: 1564 Basic stats: COMPLETE Column stats: COMPLETE
 
   Stage: Stage-0
     Fetch Operator
@@ -2149,11 +2149,11 @@ STAGE PLANS:
                         outputColumnNames: _col1
                         input vertices:
                           1 Map 4
-                        Statistics: Num rows: 400 Data size: 1600 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 391 Data size: 1564 Basic stats: COMPLETE Column stats: COMPLETE
                         Select Operator
                           expressions: _col1 (type: int)
                           outputColumnNames: _col0
-                          Statistics: Num rows: 400 Data size: 1600 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 391 Data size: 1564 Basic stats: COMPLETE Column stats: COMPLETE
                           Map Join Operator
                             condition map:
                                  Inner Join 0 to 1
@@ -2162,7 +2162,7 @@ STAGE PLANS:
                               1 _col0 (type: int)
                             input vertices:
                               0 Map 1
-                            Statistics: Num rows: 662 Data size: 5296 Basic stats: COMPLETE Column stats: COMPLETE
+                            Statistics: Num rows: 632 Data size: 5056 Basic stats: COMPLETE Column stats: COMPLETE
                             Group By Operator
                               aggregations: count()
                               mode: hash
@@ -2566,11 +2566,11 @@ STAGE PLANS:
                   0 _col0 (type: int)
                   1 _col0 (type: int)
                 outputColumnNames: _col1, _col2
-                Statistics: Num rows: 400 Data size: 38000 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 391 Data size: 37145 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col1 (type: int), substr(_col2, 5) (type: string)
                   outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 400 Data size: 38000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 391 Data size: 37145 Basic stats: COMPLETE Column stats: COMPLETE
                   Group By Operator
                     aggregations: sum(_col1)
                     keys: _col0 (type: int)
@@ -2708,11 +2708,11 @@ STAGE PLANS:
                         outputColumnNames: _col1, _col2
                         input vertices:
                           0 Map 2
-                        Statistics: Num rows: 400 Data size: 38000 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 391 Data size: 37145 Basic stats: COMPLETE Column stats: COMPLETE
                         Select Operator
                           expressions: _col1 (type: int), substr(_col2, 5) (type: string)
                           outputColumnNames: _col0, _col1
-                          Statistics: Num rows: 400 Data size: 38000 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 391 Data size: 37145 Basic stats: COMPLETE Column stats: COMPLETE
                           Group By Operator
                             aggregations: sum(_col1)
                             keys: _col0 (type: int)
@@ -2866,14 +2866,14 @@ STAGE PLANS:
                   0 _col0 (type: int)
                   1 _col1 (type: int)
                 outputColumnNames: _col1, _col2, _col3
-                Statistics: Num rows: 253 Data size: 26059 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 247 Data size: 25441 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col3 (type: int), _col2 (type: double), _col1 (type: string)
                   outputColumnNames: _col0, _col1, _col2
-                  Statistics: Num rows: 253 Data size: 26059 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 247 Data size: 25441 Basic stats: COMPLETE Column stats: COMPLETE
                   File Output Operator
                     compressed: false
-                    Statistics: Num rows: 253 Data size: 26059 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 247 Data size: 25441 Basic stats: COMPLETE Column stats: COMPLETE
                     table:
                         input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -2949,14 +2949,14 @@ STAGE PLANS:
                         outputColumnNames: _col1, _col2, _col3
                         input vertices:
                           1 Reducer 3
-                        Statistics: Num rows: 253 Data size: 26059 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 247 Data size: 25441 Basic stats: COMPLETE Column stats: COMPLETE
                         Select Operator
                           expressions: _col3 (type: int), _col2 (type: double), _col1 (type: string)
                           outputColumnNames: _col0, _col1, _col2
-                          Statistics: Num rows: 253 Data size: 26059 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 247 Data size: 25441 Basic stats: COMPLETE Column stats: COMPLETE
                           File Output Operator
                             compressed: false
-                            Statistics: Num rows: 253 Data size: 26059 Basic stats: COMPLETE Column stats: COMPLETE
+                            Statistics: Num rows: 247 Data size: 25441 Basic stats: COMPLETE Column stats: COMPLETE
                             table:
                                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -3139,14 +3139,14 @@ STAGE PLANS:
                   1 _col0 (type: int)
                   2 _col0 (type: int)
                 outputColumnNames: _col0, _col1, _col4
-                Statistics: Num rows: 633 Data size: 117738 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 619 Data size: 115134 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col0 (type: int), _col1 (type: string), _col4 (type: string)
                   outputColumnNames: _col0, _col1, _col2
-                  Statistics: Num rows: 633 Data size: 117738 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 619 Data size: 115134 Basic stats: COMPLETE Column stats: COMPLETE
                   File Output Operator
                     compressed: false
-                    Statistics: Num rows: 633 Data size: 117738 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 619 Data size: 115134 Basic stats: COMPLETE Column stats: COMPLETE
                     table:
                         input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -3202,14 +3202,14 @@ STAGE PLANS:
                         input vertices:
                           1 Map 2
                           2 Map 3
-                        Statistics: Num rows: 633 Data size: 117738 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 619 Data size: 115134 Basic stats: COMPLETE Column stats: COMPLETE
                         Select Operator
                           expressions: _col0 (type: int), _col1 (type: string), _col4 (type: string)
                           outputColumnNames: _col0, _col1, _col2
-                          Statistics: Num rows: 633 Data size: 117738 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 619 Data size: 115134 Basic stats: COMPLETE Column stats: COMPLETE
                           File Output Operator
                             compressed: false
-                            Statistics: Num rows: 633 Data size: 117738 Basic stats: COMPLETE Column stats: COMPLETE
+                            Statistics: Num rows: 619 Data size: 115134 Basic stats: COMPLETE Column stats: COMPLETE
                             table:
                                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -3304,12 +3304,12 @@ STAGE PLANS:
                         outputColumnNames: _col0, _col1
                         input vertices:
                           1 Map 3
-                        Statistics: Num rows: 400 Data size: 38000 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 391 Data size: 37145 Basic stats: COMPLETE Column stats: COMPLETE
                         Reduce Output Operator
                           key expressions: _col0 (type: int)
                           sort order: +
                           Map-reduce partition columns: _col0 (type: int)
-                          Statistics: Num rows: 400 Data size: 38000 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 391 Data size: 37145 Basic stats: COMPLETE Column stats: COMPLETE
                           value expressions: _col1 (type: string)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
@@ -3362,14 +3362,14 @@ STAGE PLANS:
                   0 _col0 (type: int)
                   1 _col0 (type: int)
                 outputColumnNames: _col0, _col1, _col3
-                Statistics: Num rows: 632 Data size: 117552 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 618 Data size: 114948 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col0 (type: int), _col1 (type: string), _col3 (type: string)
                   outputColumnNames: _col0, _col1, _col2
-                  Statistics: Num rows: 632 Data size: 117552 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 618 Data size: 114948 Basic stats: COMPLETE Column stats: COMPLETE
                   File Output Operator
                     compressed: false
-                    Statistics: Num rows: 632 Data size: 117552 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 618 Data size: 114948 Basic stats: COMPLETE Column stats: COMPLETE
                     table:
                         input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -3422,7 +3422,7 @@ STAGE PLANS:
                         outputColumnNames: _col0, _col1
                         input vertices:
                           1 Map 2
-                        Statistics: Num rows: 400 Data size: 38000 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 391 Data size: 37145 Basic stats: COMPLETE Column stats: COMPLETE
                         Map Join Operator
                           condition map:
                                Inner Join 0 to 1
@@ -3432,14 +3432,14 @@ STAGE PLANS:
                           outputColumnNames: _col0, _col1, _col3
                           input vertices:
                             1 Map 3
-                          Statistics: Num rows: 632 Data size: 117552 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 618 Data size: 114948 Basic stats: COMPLETE Column stats: COMPLETE
                           Select Operator
                             expressions: _col0 (type: int), _col1 (type: string), _col3 (type: string)
                             outputColumnNames: _col0, _col1, _col2
-                            Statistics: Num rows: 632 Data size: 117552 Basic stats: COMPLETE Column stats: COMPLETE
+                            Statistics: Num rows: 618 Data size: 114948 Basic stats: COMPLETE Column stats: COMPLETE
                             File Output Operator
                               compressed: false
-                              Statistics: Num rows: 632 Data size: 117552 Basic stats: COMPLETE Column stats: COMPLETE
+                              Statistics: Num rows: 618 Data size: 114948 Basic stats: COMPLETE Column stats: COMPLETE
                               table:
                                   input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                                   output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -3586,14 +3586,14 @@ STAGE PLANS:
                   0 _col0 (type: int)
                   1 _col0 (type: int)
                 outputColumnNames: _col0, _col1, _col3
-                Statistics: Num rows: 200 Data size: 20600 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 195 Data size: 20085 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col0 (type: int), _col1 (type: double), _col3 (type: string)
                   outputColumnNames: _col0, _col1, _col2
-                  Statistics: Num rows: 200 Data size: 20600 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 195 Data size: 20085 Basic stats: COMPLETE Column stats: COMPLETE
                   File Output Operator
                     compressed: false
-                    Statistics: Num rows: 200 Data size: 20600 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 195 Data size: 20085 Basic stats: COMPLETE Column stats: COMPLETE
                     table:
                         input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -3675,14 +3675,14 @@ STAGE PLANS:
                         outputColumnNames: _col0, _col1, _col3
                         input vertices:
                           0 Reducer 2
-                        Statistics: Num rows: 200 Data size: 20600 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 195 Data size: 20085 Basic stats: COMPLETE Column stats: COMPLETE
                         Select Operator
                           expressions: _col0 (type: int), _col1 (type: double), _col3 (type: string)
                           outputColumnNames: _col0, _col1, _col2
-                          Statistics: Num rows: 200 Data size: 20600 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 195 Data size: 20085 Basic stats: COMPLETE Column stats: COMPLETE
                           File Output Operator
                             compressed: false
-                            Statistics: Num rows: 200 Data size: 20600 Basic stats: COMPLETE Column stats: COMPLETE
+                            Statistics: Num rows: 195 Data size: 20085 Basic stats: COMPLETE Column stats: COMPLETE
                             table:
                                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -3799,14 +3799,14 @@ STAGE PLANS:
                   0 _col0 (type: int)
                   1 _col0 (type: int)
                 outputColumnNames: _col0, _col1, _col3
-                Statistics: Num rows: 253 Data size: 26059 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 247 Data size: 25441 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col0 (type: int), _col1 (type: double), _col3 (type: string)
                   outputColumnNames: _col0, _col1, _col2
-                  Statistics: Num rows: 253 Data size: 26059 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 247 Data size: 25441 Basic stats: COMPLETE Column stats: COMPLETE
                   File Output Operator
                     compressed: false
-                    Statistics: Num rows: 253 Data size: 26059 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 247 Data size: 25441 Basic stats: COMPLETE Column stats: COMPLETE
                     table:
                         input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -3882,14 +3882,14 @@ STAGE PLANS:
                         outputColumnNames: _col0, _col1, _col3
                         input vertices:
                           0 Reducer 2
-                        Statistics: Num rows: 253 Data size: 26059 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 247 Data size: 25441 Basic stats: COMPLETE Column stats: COMPLETE
                         Select Operator
                           expressions: _col0 (type: int), _col1 (type: double), _col3 (type: string)
                           outputColumnNames: _col0, _col1, _col2
-                          Statistics: Num rows: 253 Data size: 26059 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 247 Data size: 25441 Basic stats: COMPLETE Column stats: COMPLETE
                           File Output Operator
                             compressed: false
-                            Statistics: Num rows: 253 Data size: 26059 Basic stats: COMPLETE Column stats: COMPLETE
+                            Statistics: Num rows: 247 Data size: 25441 Basic stats: COMPLETE Column stats: COMPLETE
                             table:
                                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -4178,14 +4178,14 @@ STAGE PLANS:
                   0 _col0 (type: int)
                   1 _col0 (type: int)
                 outputColumnNames: _col0, _col1, _col3
-                Statistics: Num rows: 400 Data size: 74400 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 391 Data size: 72726 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col0 (type: int), _col1 (type: string), _col3 (type: string)
                   outputColumnNames: _col0, _col1, _col2
-                  Statistics: Num rows: 400 Data size: 74400 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 391 Data size: 72726 Basic stats: COMPLETE Column stats: COMPLETE
                   File Output Operator
                     compressed: false
-                    Statistics: Num rows: 400 Data size: 74400 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 391 Data size: 72726 Basic stats: COMPLETE Column stats: COMPLETE
                     table:
                         input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -4258,14 +4258,14 @@ STAGE PLANS:
                         outputColumnNames: _col0, _col1, _col3
                         input vertices:
                           0 Map 1
-                        Statistics: Num rows: 400 Data size: 74400 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 391 Data size: 72726 Basic stats: COMPLETE Column stats: COMPLETE
                         Select Operator
                           expressions: _col0 (type: int), _col1 (type: string), _col3 (type: string)
                           outputColumnNames: _col0, _col1, _col2
-                          Statistics: Num rows: 400 Data size: 74400 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 391 Data size: 72726 Basic stats: COMPLETE Column stats: COMPLETE
                           File Output Operator
                             compressed: false
-                            Statistics: Num rows: 400 Data size: 74400 Basic stats: COMPLETE Column stats: COMPLETE
+                            Statistics: Num rows: 391 Data size: 72726 Basic stats: COMPLETE Column stats: COMPLETE
                             table:
                                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -4292,7 +4292,8 @@ STAGE PLANS:
     Tez
 #### A masked pattern was here ####
       Edges:
-        Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 3 (SIMPLE_EDGE), Map 4 (BROADCAST_EDGE)
+        Map 1 <- Map 3 (BROADCAST_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
@@ -4307,15 +4308,44 @@ STAGE PLANS:
                       expressions: key (type: int), value (type: string)
                       outputColumnNames: _col0, _col1
                       Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
+                      Map Join Operator
+                        condition map:
+                             Inner Join 0 to 1
+                        keys:
+                          0 _col0 (type: int)
+                          1 _col0 (type: int)
+                        outputColumnNames: _col0, _col1
+                        input vertices:
+                          1 Map 3
+                        Statistics: Num rows: 809 Data size: 76855 Basic stats: COMPLETE Column stats: COMPLETE
+                        Reduce Output Operator
+                          key expressions: _col1 (type: string)
+                          sort order: +
+                          Map-reduce partition columns: _col1 (type: string)
+                          Statistics: Num rows: 809 Data size: 76855 Basic stats: COMPLETE Column stats: COMPLETE
+                          value expressions: _col0 (type: int)
+            Execution mode: vectorized, llap
+            LLAP IO: no inputs
+        Map 3 
+            Map Operator Tree:
+                TableScan
+                  alias: c
+                  Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
+                    Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: key (type: int)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
-                        key expressions: _col1 (type: string)
+                        key expressions: _col0 (type: int)
                         sort order: +
-                        Map-reduce partition columns: _col1 (type: string)
-                        Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
-                        value expressions: _col0 (type: int)
+                        Map-reduce partition columns: _col0 (type: int)
+                        Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
-        Map 3 
+        Map 4 
             Map Operator Tree:
                 TableScan
                   alias: b
@@ -4335,25 +4365,6 @@ STAGE PLANS:
                         value expressions: _col0 (type: int)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
-        Map 4 
-            Map Operator Tree:
-                TableScan
-                  alias: c
-                  Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
-                  Filter Operator
-                    predicate: key is not null (type: boolean)
-                    Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
-                    Select Operator
-                      expressions: key (type: int)
-                      outputColumnNames: _col0
-                      Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
-                      Reduce Output Operator
-                        key expressions: _col0 (type: int)
-                        sort order: +
-                        Map-reduce partition columns: _col0 (type: int)
-                        Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
         Reducer 2 
             Execution mode: llap
             Reduce Operator Tree:
@@ -4363,29 +4374,19 @@ STAGE PLANS:
                 keys:
                   0 _col1 (type: string)
                   1 _col1 (type: string)
-                outputColumnNames: _col0, _col2
-                Statistics: Num rows: 809 Data size: 6472 Basic stats: COMPLETE Column stats: COMPLETE
-                Map Join Operator
-                  condition map:
-                       Inner Join 0 to 1
-                  keys:
-                    0 _col0 (type: int)
-                    1 _col0 (type: int)
-                  outputColumnNames: _col0, _col2
-                  input vertices:
-                    1 Map 4
-                  Statistics: Num rows: 1339 Data size: 10712 Basic stats: COMPLETE Column stats: COMPLETE
-                  Select Operator
-                    expressions: _col0 (type: int), _col2 (type: int)
-                    outputColumnNames: _col0, _col1
-                    Statistics: Num rows: 1339 Data size: 10712 Basic stats: COMPLETE Column stats: COMPLETE
-                    File Output Operator
-                      compressed: false
-                      Statistics: Num rows: 1339 Data size: 10712 Basic stats: COMPLETE Column stats: COMPLETE
-                      table:
-                          input format: org.apache.hadoop.mapred.SequenceFileInputFormat
-                          output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
-                          serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                outputColumnNames: _col0, _col3
+                Statistics: Num rows: 1309 Data size: 10472 Basic stats: COMPLETE Column stats: COMPLETE
+                Select Operator
+                  expressions: _col0 (type: int), _col3 (type: int)
+                  outputColumnNames: _col0, _col1
+                  Statistics: Num rows: 1309 Data size: 10472 Basic stats: COMPLETE Column stats: COMPLETE
+                  File Output Operator
+                    compressed: false
+                    Statistics: Num rows: 1309 Data size: 10472 Basic stats: COMPLETE Column stats: COMPLETE
+                    table:
+                        input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                        output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                        serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
 
   Stage: Stage-0
     Fetch Operator
@@ -4406,7 +4407,7 @@ STAGE PLANS:
     Tez
 #### A masked pattern was here ####
       Edges:
-        Map 1 <- Map 2 (CUSTOM_EDGE), Map 3 (BROADCAST_EDGE)
+        Map 1 <- Map 2 (CUSTOM_EDGE), Map 3 (CUSTOM_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
@@ -4425,29 +4426,29 @@ STAGE PLANS:
                         condition map:
                              Inner Join 0 to 1
                         keys:
-                          0 _col1 (type: string)
-                          1 _col1 (type: string)
-                        outputColumnNames: _col0, _col2
+                          0 _col0 (type: int)
+                          1 _col0 (type: int)
+                        outputColumnNames: _col0, _col1
                         input vertices:
                           1 Map 2
-                        Statistics: Num rows: 809 Data size: 6472 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 809 Data size: 76855 Basic stats: COMPLETE Column stats: COMPLETE
                         Map Join Operator
                           condition map:
                                Inner Join 0 to 1
                           keys:
-                            0 _col0 (type: int)
-                            1 _col0 (type: int)
-                          outputColumnNames: _col0, _col2
+                            0 _col1 (type: string)
+                            1 _col1 (type: string)
+                          outputColumnNames: _col0, _col3
                           input vertices:
                             1 Map 3
-                          Statistics: Num rows: 1339 Data size: 10712 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 1309 Data size: 10472 Basic stats: COMPLETE Column stats: COMPLETE
                           Select Operator
-                            expressions: _col0 (type: int), _col2 (type: int)
+                            expressions: _col0 (type: int), _col3 (type: int)
                             outputColumnNames: _col0, _col1
-                            Statistics: Num rows: 1339 Data size: 10712 Basic stats: COMPLETE Column stats: COMPLETE
+                            Statistics: Num rows: 1309 Data size: 10472 Basic stats: COMPLETE Column stats: COMPLETE
                             File Output Operator
                               compressed: false
-                              Statistics: Num rows: 1339 Data size: 10712 Basic stats: COMPLETE Column stats: COMPLETE
+                              Statistics: Num rows: 1309 Data size: 10472 Basic stats: COMPLETE Column stats: COMPLETE
                               table:
                                   input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                                   output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -4457,6 +4458,25 @@ STAGE PLANS:
         Map 2 
             Map Operator Tree:
                 TableScan
+                  alias: c
+                  Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
+                    Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: key (type: int)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
+                        Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
+            Execution mode: vectorized, llap
+            LLAP IO: no inputs
+        Map 3 
+            Map Operator Tree:
+                TableScan
                   alias: b
                   Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
@@ -4474,25 +4494,6 @@ STAGE PLANS:
                         value expressions: _col0 (type: int)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
-        Map 3 
-            Map Operator Tree:
-                TableScan
-                  alias: c
-                  Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
-                  Filter Operator
-                    predicate: key is not null (type: boolean)
-                    Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
-                    Select Operator
-                      expressions: key (type: int)
-                      outputColumnNames: _col0
-                      Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
-                      Reduce Output Operator
-                        key expressions: _col0 (type: int)
-                        sort order: +
-                        Map-reduce partition columns: _col0 (type: int)
-                        Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
 
   Stage: Stage-0
     Fetch Operator
@@ -4585,14 +4586,14 @@ STAGE PLANS:
                   0 _col0 (type: int), _col2 (type: string)
                   1 _col0 (type: int), _col2 (type: string)
                 outputColumnNames: _col0, _col1, _col4
-                Statistics: Num rows: 400 Data size: 74400 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 391 Data size: 72726 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col0 (type: int), _col1 (type: string), _col4 (type: string)
                   outputColumnNames: _col0, _col1, _col2
-                  Statistics: Num rows: 400 Data size: 74400 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 391 Data size: 72726 Basic stats: COMPLETE Column stats: COMPLETE
                   File Output Operator
                     compressed: false
-                    Statistics: Num rows: 400 Data size: 74400 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 391 Data size: 72726 Basic stats: COMPLETE Column stats: COMPLETE
                     table:
                         input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -4689,14 +4690,14 @@ STAGE PLANS:
                   0 _col0 (type: int), _col2 (type: string)
                   1 _col0 (type: int), _col2 (type: string)
                 outputColumnNames: _col0, _col1, _col4
-                Statistics: Num rows: 400 Data size: 74400 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 391 Data size: 72726 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col0 (type: int), _col1 (type: string), _col4 (type: string)
                   outputColumnNames: _col0, _col1, _col2
-                  Statistics: Num rows: 400 Data size: 74400 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 391 Data size: 72726 Basic stats: COMPLETE Column stats: COMPLETE
                   File Output Operator
                     compressed: false
-                    Statistics: Num rows: 400 Data size: 74400 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 391 Data size: 72726 Basic stats: COMPLETE Column stats: COMPLETE
                     table:
                         input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat

http://git-wip-us.apache.org/repos/asf/hive/blob/3c3880c0/ql/src/test/results/clientpositive/llap/bucket_map_join_tez2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/bucket_map_join_tez2.q.out b/ql/src/test/results/clientpositive/llap/bucket_map_join_tez2.q.out
index 51b138a..8490e3f 100644
--- a/ql/src/test/results/clientpositive/llap/bucket_map_join_tez2.q.out
+++ b/ql/src/test/results/clientpositive/llap/bucket_map_join_tez2.q.out
@@ -178,7 +178,7 @@ STAGE PLANS:
 #### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE)
-        Reducer 3 <- Map 5 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
+        Reducer 3 <- Map 1 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
@@ -194,18 +194,11 @@ STAGE PLANS:
                       outputColumnNames: _col0, _col1
                       Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
-                        key expressions: _col1 (type: string)
+                        key expressions: _col0 (type: int)
                         sort order: +
-                        Map-reduce partition columns: _col1 (type: string)
+                        Map-reduce partition columns: _col0 (type: int)
                         Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
-                        value expressions: _col0 (type: int)
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
-        Map 4 
-            Map Operator Tree:
-                TableScan
-                  alias: b
-                  Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
+                        value expressions: _col1 (type: string)
                   Filter Operator
                     predicate: value is not null (type: boolean)
                     Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
@@ -221,7 +214,7 @@ STAGE PLANS:
                         value expressions: _col0 (type: int)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
-        Map 5 
+        Map 4 
             Map Operator Tree:
                 TableScan
                   alias: c
@@ -247,16 +240,16 @@ STAGE PLANS:
                 condition map:
                      Inner Join 0 to 1
                 keys:
-                  0 _col1 (type: string)
-                  1 _col1 (type: string)
-                outputColumnNames: _col0, _col2
-                Statistics: Num rows: 809 Data size: 6472 Basic stats: COMPLETE Column stats: COMPLETE
+                  0 _col0 (type: int)
+                  1 _col0 (type: int)
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 809 Data size: 76855 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
-                  key expressions: _col0 (type: int)
+                  key expressions: _col1 (type: string)
                   sort order: +
-                  Map-reduce partition columns: _col0 (type: int)
-                  Statistics: Num rows: 809 Data size: 6472 Basic stats: COMPLETE Column stats: COMPLETE
-                  value expressions: _col2 (type: int)
+                  Map-reduce partition columns: _col1 (type: string)
+                  Statistics: Num rows: 809 Data size: 76855 Basic stats: COMPLETE Column stats: COMPLETE
+                  value expressions: _col0 (type: int)
         Reducer 3 
             Execution mode: llap
             Reduce Operator Tree:
@@ -264,17 +257,17 @@ STAGE PLANS:
                 condition map:
                      Inner Join 0 to 1
                 keys:
-                  0 _col0 (type: int)
-                  1 _col0 (type: int)
-                outputColumnNames: _col0, _col2
-                Statistics: Num rows: 1339 Data size: 10712 Basic stats: COMPLETE Column stats: COMPLETE
+                  0 _col1 (type: string)
+                  1 _col1 (type: string)
+                outputColumnNames: _col0, _col3
+                Statistics: Num rows: 1309 Data size: 10472 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
-                  expressions: _col0 (type: int), _col2 (type: int)
+                  expressions: _col0 (type: int), _col3 (type: int)
                   outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 1339 Data size: 10712 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 1309 Data size: 10472 Basic stats: COMPLETE Column stats: COMPLETE
                   File Output Operator
                     compressed: false
-                    Statistics: Num rows: 1339 Data size: 10712 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 1309 Data size: 10472 Basic stats: COMPLETE Column stats: COMPLETE
                     table:
                         input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -299,8 +292,8 @@ STAGE PLANS:
     Tez
 #### A masked pattern was here ####
       Edges:
+        Map 1 <- Map 3 (CUSTOM_EDGE)
         Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE)
-        Reducer 3 <- Map 5 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
@@ -315,12 +308,41 @@ STAGE PLANS:
                       expressions: key (type: int), value (type: string)
                       outputColumnNames: _col0, _col1
                       Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
+                      Map Join Operator
+                        condition map:
+                             Inner Join 0 to 1
+                        keys:
+                          0 _col0 (type: int)
+                          1 _col0 (type: int)
+                        outputColumnNames: _col0, _col1
+                        input vertices:
+                          1 Map 3
+                        Statistics: Num rows: 809 Data size: 76855 Basic stats: COMPLETE Column stats: COMPLETE
+                        Reduce Output Operator
+                          key expressions: _col1 (type: string)
+                          sort order: +
+                          Map-reduce partition columns: _col1 (type: string)
+                          Statistics: Num rows: 809 Data size: 76855 Basic stats: COMPLETE Column stats: COMPLETE
+                          value expressions: _col0 (type: int)
+            Execution mode: vectorized, llap
+            LLAP IO: no inputs
+        Map 3 
+            Map Operator Tree:
+                TableScan
+                  alias: c
+                  Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
+                    Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: key (type: int)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
-                        key expressions: _col1 (type: string)
+                        key expressions: _col0 (type: int)
                         sort order: +
-                        Map-reduce partition columns: _col1 (type: string)
-                        Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
-                        value expressions: _col0 (type: int)
+                        Map-reduce partition columns: _col0 (type: int)
+                        Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Map 4 
@@ -343,25 +365,6 @@ STAGE PLANS:
                         value expressions: _col0 (type: int)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
-        Map 5 
-            Map Operator Tree:
-                TableScan
-                  alias: c
-                  Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
-                  Filter Operator
-                    predicate: key is not null (type: boolean)
-                    Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
-                    Select Operator
-                      expressions: key (type: int)
-                      outputColumnNames: _col0
-                      Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
-                      Reduce Output Operator
-                        key expressions: _col0 (type: int)
-                        sort order: +
-                        Map-reduce partition columns: _col0 (type: int)
-                        Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
         Reducer 2 
             Execution mode: llap
             Reduce Operator Tree:
@@ -371,32 +374,15 @@ STAGE PLANS:
                 keys:
                   0 _col1 (type: string)
                   1 _col1 (type: string)
-                outputColumnNames: _col0, _col2
-                Statistics: Num rows: 809 Data size: 6472 Basic stats: COMPLETE Column stats: COMPLETE
-                Reduce Output Operator
-                  key expressions: _col0 (type: int)
-                  sort order: +
-                  Map-reduce partition columns: _col0 (type: int)
-                  Statistics: Num rows: 809 Data size: 6472 Basic stats: COMPLETE Column stats: COMPLETE
-                  value expressions: _col2 (type: int)
-        Reducer 3 
-            Execution mode: llap
-            Reduce Operator Tree:
-              Merge Join Operator
-                condition map:
-                     Inner Join 0 to 1
-                keys:
-                  0 _col0 (type: int)
-                  1 _col0 (type: int)
-                outputColumnNames: _col0, _col2
-                Statistics: Num rows: 1339 Data size: 10712 Basic stats: COMPLETE Column stats: COMPLETE
+                outputColumnNames: _col0, _col3
+                Statistics: Num rows: 1309 Data size: 10472 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
-                  expressions: _col0 (type: int), _col2 (type: int)
+                  expressions: _col0 (type: int), _col3 (type: int)
                   outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 1339 Data size: 10712 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 1309 Data size: 10472 Basic stats: COMPLETE Column stats: COMPLETE
                   File Output Operator
                     compressed: false
-                    Statistics: Num rows: 1339 Data size: 10712 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 1309 Data size: 10472 Basic stats: COMPLETE Column stats: COMPLETE
                     table:
                         input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -684,10 +670,10 @@ STAGE PLANS:
                   0 _col0 (type: int)
                   1 _col0 (type: int)
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 272 Data size: 2176 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 267 Data size: 2136 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 272 Data size: 2176 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 267 Data size: 2136 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -738,10 +724,10 @@ STAGE PLANS:
                         outputColumnNames: _col0, _col1
                         input vertices:
                           1 Map 2
-                        Statistics: Num rows: 272 Data size: 2176 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 267 Data size: 2136 Basic stats: COMPLETE Column stats: COMPLETE
                         File Output Operator
                           compressed: false
-                          Statistics: Num rows: 272 Data size: 2176 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 267 Data size: 2136 Basic stats: COMPLETE Column stats: COMPLETE
                           table:
                               input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                               output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -840,10 +826,10 @@ STAGE PLANS:
                   0 _col0 (type: int)
                   1 _col0 (type: int)
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 272 Data size: 2176 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 267 Data size: 2136 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 272 Data size: 2176 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 267 Data size: 2136 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -894,10 +880,10 @@ STAGE PLANS:
                         outputColumnNames: _col0, _col1
                         input vertices:
                           1 Map 2
-                        Statistics: Num rows: 272 Data size: 2176 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 267 Data size: 2136 Basic stats: COMPLETE Column stats: COMPLETE
                         File Output Operator
                           compressed: false
-                          Statistics: Num rows: 272 Data size: 2176 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 267 Data size: 2136 Basic stats: COMPLETE Column stats: COMPLETE
                           table:
                               input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                               output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -996,10 +982,10 @@ STAGE PLANS:
                   0 _col0 (type: int)
                   1 _col0 (type: int)
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 272 Data size: 2176 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 267 Data size: 2136 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 272 Data size: 2176 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 267 Data size: 2136 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -1069,10 +1055,10 @@ STAGE PLANS:
                         outputColumnNames: _col0, _col1
                         input vertices:
                           0 Map 1
-                        Statistics: Num rows: 272 Data size: 2176 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 267 Data size: 2136 Basic stats: COMPLETE Column stats: COMPLETE
                         File Output Operator
                           compressed: false
-                          Statistics: Num rows: 272 Data size: 2176 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 267 Data size: 2136 Basic stats: COMPLETE Column stats: COMPLETE
                           table:
                               input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                               output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat


[13/50] [abbrv] hive git commit: HIVE-19248: REPL LOAD couldn't copy file from source CM path and also doesn't throw error if file copy fails (Sankar Hariappan, reviewed by Mahesh Kumar Behera, Thejas M Nair)

Posted by vg...@apache.org.
HIVE-19248: REPL LOAD couldn't copy file from source CM path and also doesn't throw error if file copy fails (Sankar Hariappan, reviewed by Mahesh Kumar Behera, Thejas M Nair)


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

Branch: refs/heads/branch-3.0.0
Commit: a8fc0e67183ed063bb2b3aee69a290ac734b3f51
Parents: 3102737
Author: Sankar Hariappan <sa...@apache.org>
Authored: Thu May 10 00:04:52 2018 +0530
Committer: Sankar Hariappan <sa...@apache.org>
Committed: Thu May 10 00:04:52 2018 +0530

----------------------------------------------------------------------
 .../listener/DbNotificationListener.java        |  19 +--
 .../hive/metastore/TestReplChangeManager.java   |  54 +++---
 .../hadoop/hive/ql/exec/ReplCopyTask.java       |  12 +-
 .../hadoop/hive/ql/parse/repl/CopyUtils.java    | 168 ++++++++++++-------
 .../hive/ql/parse/repl/load/DumpMetaData.java   |   1 -
 .../load/message/CreateFunctionHandler.java     |   4 +-
 .../apache/hadoop/hive/shims/Hadoop23Shims.java |   3 +-
 .../hadoop/hive/shims/TestHadoop23Shims.java    |  24 +--
 .../hive/metastore/ReplChangeManager.java       |  94 ++++++-----
 .../hadoop/hive/metastore/utils/FileUtils.java  |  64 ++++++-
 .../hadoop/hive/metastore/utils/HdfsUtils.java  |   3 +-
 .../hive/metastore/utils/StringUtils.java       |  24 ++-
 12 files changed, 302 insertions(+), 168 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/a8fc0e67/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/listener/DbNotificationListener.java
----------------------------------------------------------------------
diff --git a/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/listener/DbNotificationListener.java b/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/listener/DbNotificationListener.java
index 7835691..6321f9b 100644
--- a/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/listener/DbNotificationListener.java
+++ b/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/listener/DbNotificationListener.java
@@ -426,7 +426,14 @@ public class DbNotificationListener extends TransactionalMetaStoreEventListener
 
     @Override
     public String next() {
-      String result = encodeFileUri(files.get(i), chksums != null? chksums.get(i) : null);
+      String result;
+      try {
+        result = ReplChangeManager.encodeFileUri(files.get(i), chksums != null ? chksums.get(i) : null, null);
+      } catch (IOException e) {
+        // File operations failed
+        LOG.error("Encoding file URI failed with error " + e.getMessage());
+        throw new RuntimeException(e.getMessage());
+      }
       i++;
       return result;
     }
@@ -788,14 +795,4 @@ public class DbNotificationListener extends TransactionalMetaStoreEventListener
     }
 
   }
-
-  // TODO: this needs to be enhanced once change management based filesystem is implemented
-  // Currently using fileuri#checksum as the format
-  private String encodeFileUri(String fileUriStr, String fileChecksum) {
-    if (fileChecksum != null) {
-      return fileUriStr + "#" + fileChecksum;
-    } else {
-      return fileUriStr;
-    }
-  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/a8fc0e67/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestReplChangeManager.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestReplChangeManager.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestReplChangeManager.java
index 6ade76d..e63250c 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestReplChangeManager.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestReplChangeManager.java
@@ -175,19 +175,19 @@ public class TestReplChangeManager {
     // verify cm.recycle(db, table, part) api moves file to cmroot dir
     int ret = cm.recycle(part1Path, RecycleType.MOVE, false);
     Assert.assertEquals(ret, 1);
-    Path cmPart1Path = ReplChangeManager.getCMPath(hiveConf, part1Path.getName(), path1Chksum);
+    Path cmPart1Path = ReplChangeManager.getCMPath(hiveConf, part1Path.getName(), path1Chksum, cmroot.toString());
     assertTrue(cmPart1Path.getFileSystem(hiveConf).exists(cmPart1Path));
 
     // Verify dropPartition recycle part files
     client.dropPartition(dbName, tblName, Arrays.asList("20160102"));
     assertFalse(part2Path.getFileSystem(hiveConf).exists(part2Path));
-    Path cmPart2Path = ReplChangeManager.getCMPath(hiveConf, part2Path.getName(), path2Chksum);
+    Path cmPart2Path = ReplChangeManager.getCMPath(hiveConf, part2Path.getName(), path2Chksum, cmroot.toString());
     assertTrue(cmPart2Path.getFileSystem(hiveConf).exists(cmPart2Path));
 
     // Verify dropTable recycle partition files
     client.dropTable(dbName, tblName);
     assertFalse(part3Path.getFileSystem(hiveConf).exists(part3Path));
-    Path cmPart3Path = ReplChangeManager.getCMPath(hiveConf, part3Path.getName(), path3Chksum);
+    Path cmPart3Path = ReplChangeManager.getCMPath(hiveConf, part3Path.getName(), path3Chksum, cmroot.toString());
     assertTrue(cmPart3Path.getFileSystem(hiveConf).exists(cmPart3Path));
 
     client.dropDatabase(dbName, true, true);
@@ -246,17 +246,17 @@ public class TestReplChangeManager {
     cm.recycle(filePath1, RecycleType.MOVE, false);
     assertFalse(filePath1.getFileSystem(hiveConf).exists(filePath1));
 
-    Path cmPath1 = ReplChangeManager.getCMPath(hiveConf, filePath1.getName(), fileChksum1);
+    Path cmPath1 = ReplChangeManager.getCMPath(hiveConf, filePath1.getName(), fileChksum1, cmroot.toString());
     assertTrue(cmPath1.getFileSystem(hiveConf).exists(cmPath1));
 
     // Verify dropTable recycle table files
     client.dropTable(dbName, tblName);
 
-    Path cmPath2 = ReplChangeManager.getCMPath(hiveConf, filePath2.getName(), fileChksum2);
+    Path cmPath2 = ReplChangeManager.getCMPath(hiveConf, filePath2.getName(), fileChksum2,cmroot.toString());
     assertFalse(filePath2.getFileSystem(hiveConf).exists(filePath2));
     assertTrue(cmPath2.getFileSystem(hiveConf).exists(cmPath2));
 
-    Path cmPath3 = ReplChangeManager.getCMPath(hiveConf, filePath3.getName(), fileChksum3);
+    Path cmPath3 = ReplChangeManager.getCMPath(hiveConf, filePath3.getName(), fileChksum3, cmroot.toString());
     assertFalse(filePath3.getFileSystem(hiveConf).exists(filePath3));
     assertTrue(cmPath3.getFileSystem(hiveConf).exists(cmPath3));
 
@@ -298,17 +298,21 @@ public class TestReplChangeManager {
     ReplChangeManager.getInstance(hiveConf).recycle(dirTbl2, RecycleType.MOVE, false);
     ReplChangeManager.getInstance(hiveConf).recycle(dirTbl3, RecycleType.MOVE, true);
 
-    assertTrue(fs.exists(ReplChangeManager.getCMPath(hiveConf, part11.getName(), fileChksum11)));
-    assertTrue(fs.exists(ReplChangeManager.getCMPath(hiveConf, part12.getName(), fileChksum12)));
-    assertTrue(fs.exists(ReplChangeManager.getCMPath(hiveConf, part21.getName(), fileChksum21)));
-    assertTrue(fs.exists(ReplChangeManager.getCMPath(hiveConf, part22.getName(), fileChksum22)));
-    assertTrue(fs.exists(ReplChangeManager.getCMPath(hiveConf, part31.getName(), fileChksum31)));
-    assertTrue(fs.exists(ReplChangeManager.getCMPath(hiveConf, part32.getName(), fileChksum32)));
-
-    fs.setTimes(ReplChangeManager.getCMPath(hiveConf, part11.getName(), fileChksum11), now - 86400*1000*2, now - 86400*1000*2);
-    fs.setTimes(ReplChangeManager.getCMPath(hiveConf, part21.getName(), fileChksum21), now - 86400*1000*2, now - 86400*1000*2);
-    fs.setTimes(ReplChangeManager.getCMPath(hiveConf, part31.getName(), fileChksum31), now - 86400*1000*2, now - 86400*1000*2);
-    fs.setTimes(ReplChangeManager.getCMPath(hiveConf, part32.getName(), fileChksum32), now - 86400*1000*2, now - 86400*1000*2);
+    assertTrue(fs.exists(ReplChangeManager.getCMPath(hiveConf, part11.getName(), fileChksum11, cmroot.toString())));
+    assertTrue(fs.exists(ReplChangeManager.getCMPath(hiveConf, part12.getName(), fileChksum12, cmroot.toString())));
+    assertTrue(fs.exists(ReplChangeManager.getCMPath(hiveConf, part21.getName(), fileChksum21, cmroot.toString())));
+    assertTrue(fs.exists(ReplChangeManager.getCMPath(hiveConf, part22.getName(), fileChksum22, cmroot.toString())));
+    assertTrue(fs.exists(ReplChangeManager.getCMPath(hiveConf, part31.getName(), fileChksum31, cmroot.toString())));
+    assertTrue(fs.exists(ReplChangeManager.getCMPath(hiveConf, part32.getName(), fileChksum32, cmroot.toString())));
+
+    fs.setTimes(ReplChangeManager.getCMPath(hiveConf, part11.getName(), fileChksum11, cmroot.toString()),
+            now - 86400*1000*2, now - 86400*1000*2);
+    fs.setTimes(ReplChangeManager.getCMPath(hiveConf, part21.getName(), fileChksum21, cmroot.toString()),
+            now - 86400*1000*2, now - 86400*1000*2);
+    fs.setTimes(ReplChangeManager.getCMPath(hiveConf, part31.getName(), fileChksum31, cmroot.toString()),
+            now - 86400*1000*2, now - 86400*1000*2);
+    fs.setTimes(ReplChangeManager.getCMPath(hiveConf, part32.getName(), fileChksum32, cmroot.toString()),
+            now - 86400*1000*2, now - 86400*1000*2);
 
     ReplChangeManager.scheduleCMClearer(hiveConf);
 
@@ -321,12 +325,12 @@ public class TestReplChangeManager {
       if (end - start > 5000) {
         Assert.fail("timeout, cmroot has not been cleared");
       }
-      if (!fs.exists(ReplChangeManager.getCMPath(hiveConf, part11.getName(), fileChksum11)) &&
-          fs.exists(ReplChangeManager.getCMPath(hiveConf, part12.getName(), fileChksum12)) &&
-          !fs.exists(ReplChangeManager.getCMPath(hiveConf, part21.getName(), fileChksum21)) &&
-          fs.exists(ReplChangeManager.getCMPath(hiveConf, part22.getName(), fileChksum22)) &&
-          !fs.exists(ReplChangeManager.getCMPath(hiveConf, part31.getName(), fileChksum31)) &&
-          !fs.exists(ReplChangeManager.getCMPath(hiveConf, part32.getName(), fileChksum32))) {
+      if (!fs.exists(ReplChangeManager.getCMPath(hiveConf, part11.getName(), fileChksum11, cmroot.toString())) &&
+          fs.exists(ReplChangeManager.getCMPath(hiveConf, part12.getName(), fileChksum12, cmroot.toString())) &&
+          !fs.exists(ReplChangeManager.getCMPath(hiveConf, part21.getName(), fileChksum21, cmroot.toString())) &&
+          fs.exists(ReplChangeManager.getCMPath(hiveConf, part22.getName(), fileChksum22, cmroot.toString())) &&
+          !fs.exists(ReplChangeManager.getCMPath(hiveConf, part31.getName(), fileChksum31, cmroot.toString())) &&
+          !fs.exists(ReplChangeManager.getCMPath(hiveConf, part32.getName(), fileChksum32, cmroot.toString()))) {
         cleared = true;
       }
     } while (!cleared);
@@ -335,8 +339,8 @@ public class TestReplChangeManager {
   @Test
   public void shouldIdentifyCMURIs() {
     assertTrue(ReplChangeManager
-        .isCMFileUri(new Path("hdfs://localhost:90000/somepath/adir/", "ab.jar#e239s2233"), fs));
+        .isCMFileUri(new Path("hdfs://localhost:90000/somepath/adir/", "ab.jar#e239s2233")));
     assertFalse(ReplChangeManager
-        .isCMFileUri(new Path("/somepath/adir/", "ab.jar"), fs));
+        .isCMFileUri(new Path("/somepath/adir/", "ab.jar")));
   }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/a8fc0e67/ql/src/java/org/apache/hadoop/hive/ql/exec/ReplCopyTask.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/ReplCopyTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/ReplCopyTask.java
index 4f38a5f..5fecf81 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/ReplCopyTask.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/ReplCopyTask.java
@@ -80,10 +80,10 @@ public class ReplCopyTask extends Task<ReplCopyWork> implements Serializable {
 
       // This should only be true for copy tasks created from functions, otherwise there should never
       // be a CM uri in the from path.
-      if (ReplChangeManager.isCMFileUri(fromPath, srcFs)) {
-        String[] result = ReplChangeManager.getFileWithChksumFromURI(fromPath.toString());
+      if (ReplChangeManager.isCMFileUri(fromPath)) {
+        String[] result = ReplChangeManager.decodeFileUri(fromPath.toString());
         ReplChangeManager.FileInfo sourceInfo = ReplChangeManager
-            .getFileInfo(new Path(result[0]), result[1], result[2], conf);
+            .getFileInfo(new Path(result[0]), result[1], result[2], result[3], conf);
         if (FileUtils.copy(
             sourceInfo.getSrcFs(), sourceInfo.getSourcePath(),
             dstFs, toPath, false, false, conf)) {
@@ -187,14 +187,14 @@ public class ReplCopyTask extends Task<ReplCopyWork> implements Serializable {
       while ((line = br.readLine()) != null) {
         LOG.debug("ReplCopyTask :_filesReadLine: {}", line);
 
-        String[] fileWithChksum = ReplChangeManager.getFileWithChksumFromURI(line);
+        String[] fragments = ReplChangeManager.decodeFileUri(line);
         try {
           ReplChangeManager.FileInfo f = ReplChangeManager
-              .getFileInfo(new Path(fileWithChksum[0]), fileWithChksum[1], fileWithChksum[2], conf);
+              .getFileInfo(new Path(fragments[0]), fragments[1], fragments[2], fragments[3], conf);
           filePaths.add(f);
         } catch (MetaException e) {
           // issue warning for missing file and throw exception
-          LOG.warn("Cannot find {} in source repo or cmroot", fileWithChksum[0]);
+          LOG.warn("Cannot find {} in source repo or cmroot", fragments[0]);
           throw new IOException(e.getMessage());
         }
         // Note - we need srcFs rather than fs, because it is possible that the _files lists files

http://git-wip-us.apache.org/repos/asf/hive/blob/a8fc0e67/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/CopyUtils.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/CopyUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/CopyUtils.java
index 529ea21..2557121 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/CopyUtils.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/CopyUtils.java
@@ -47,7 +47,7 @@ public class CopyUtils {
   private static final Logger LOG = LoggerFactory.getLogger(CopyUtils.class);
   // https://hadoop.apache.org/docs/stable/hadoop-project-dist/hadoop-hdfs/TransparentEncryption.html#Running_as_the_superuser
   private static final String RAW_RESERVED_VIRTUAL_PATH = "/.reserved/raw/";
-  private static final int MAX_COPY_RETRY = 3;
+  private static final int MAX_COPY_RETRY = 5;
 
   private final HiveConf hiveConf;
   private final long maxCopyFileSize;
@@ -82,71 +82,126 @@ public class CopyUtils {
           LOG.error("Failed to create destination directory: " + destination);
           throw new IOException("Destination directory creation failed");
         }
-        doCopyRetry(sourceFs, fileInfoList, destinationFs, destination, useRegularCopy);
 
-        // Verify checksum, retry if checksum changed
-        List<ReplChangeManager.FileInfo> retryFileInfoList = new ArrayList<>();
-        for (ReplChangeManager.FileInfo srcFile : srcFiles) {
-          if (!srcFile.isUseSourcePath()) {
-            // If already use cmpath, nothing we can do here, skip this file
-            continue;
-          }
-          String sourceChecksumString = srcFile.getCheckSum();
-          if (sourceChecksumString != null) {
-            String verifySourceChecksumString;
-            try {
-              verifySourceChecksumString
-                      = ReplChangeManager.checksumFor(srcFile.getSourcePath(), sourceFs);
-            } catch (IOException e) {
-              // Retry with CM path
-              verifySourceChecksumString = null;
-            }
-            if ((verifySourceChecksumString == null)
-                    || !sourceChecksumString.equals(verifySourceChecksumString)) {
-              // If checksum does not match, likely the file is changed/removed, copy again from cm
-              srcFile.setIsUseSourcePath(false);
-              retryFileInfoList.add(srcFile);
-            }
-          }
-        }
-        if (!retryFileInfoList.isEmpty()) {
-          doCopyRetry(sourceFs, retryFileInfoList, destinationFs, destination, useRegularCopy);
-        }
+		    // Copy files with retry logic on failure or source file is dropped or changed.
+        doCopyRetry(sourceFs, fileInfoList, destinationFs, destination, useRegularCopy);
       }
     }
   }
 
-  private void doCopyRetry(FileSystem sourceFs, List<ReplChangeManager.FileInfo> fileList,
+  private void doCopyRetry(FileSystem sourceFs, List<ReplChangeManager.FileInfo> srcFileList,
                            FileSystem destinationFs, Path destination,
                            boolean useRegularCopy) throws IOException, LoginException {
     int repeat = 0;
-    List<Path> pathList = Lists.transform(fileList, ReplChangeManager.FileInfo::getEffectivePath);
+    boolean isCopyError = false;
+    List<Path> pathList = Lists.transform(srcFileList, ReplChangeManager.FileInfo::getEffectivePath);
     while (!pathList.isEmpty() && (repeat < MAX_COPY_RETRY)) {
+      LOG.info("Attempt: " + (repeat+1) + ". Copying files: " + pathList);
       try {
+        isCopyError = false;
         doCopyOnce(sourceFs, pathList, destinationFs, destination, useRegularCopy);
-        return;
       } catch (IOException e) {
-        pathList = new ArrayList<>();
+        // If copy fails, fall through the retry logic
+        isCopyError = true;
+      }
+      pathList = getFilesToRetry(sourceFs, srcFileList, destinationFs, destination, isCopyError);
+      repeat++;
+    }
+
+    // If still files remains to be copied due to failure/checksum mismatch after several attempts, then throw error
+    if (!pathList.isEmpty()) {
+      LOG.error("File copy failed even after several attempts. Files list: " + srcFileList);
+      throw new IOException("File copy failed even after several attempts.");
+    }
+  }
+
+  // Traverse through all the source files and see if any file is not copied or partially copied.
+  // If yes, then add to the retry list. If source file missing, then retry with CM path. if CM path
+  // itself is missing, then throw error.
+  private List<Path> getFilesToRetry(FileSystem sourceFs, List<ReplChangeManager.FileInfo> srcFileList,
+                                     FileSystem destinationFs, Path destination, boolean isCopyError)
+          throws IOException {
+    List<Path> pathList = new ArrayList<Path>();
+
+    // Going through file list and make the retry list
+    for (ReplChangeManager.FileInfo srcFile : srcFileList) {
+      if (srcFile.isCopyDone()) {
+        // If already copied successfully, ignore it.
+        continue;
+      }
+      Path srcPath = srcFile.getEffectivePath();
+      Path destPath = new Path(destination, srcPath.getName());
+      if (destinationFs.exists(destPath)) {
+        // If destination file is present and checksum of source mismatch, then retry copy.
+        if (isSourceFileMismatch(sourceFs, srcFile)) {
+          // Delete the incorrectly copied file and retry with CM path
+          destinationFs.delete(destPath, true);
+          srcFile.setIsUseSourcePath(false);
+        } else {
+          // If the retry logic is reached after copy error, then include the copied file as well.
+          // This is needed as we cannot figure out which file is incorrectly copied.
+          // Expecting distcp to skip the properly copied file based on CRC check or copy it if CRC mismatch.
 
-        // Going through file list, retry with CM if applicable
-        for (ReplChangeManager.FileInfo file : fileList) {
-          Path copyPath = file.getEffectivePath();
-          if (!destinationFs.exists(new Path(destination, copyPath.getName()))) {
-            if (!sourceFs.exists(copyPath)) {
-              if (file.isUseSourcePath()) {
-                // Source file missing, then try with CM path
-                file.setIsUseSourcePath(false);
-              } else {
-                // CM path itself is missing, so, cannot recover from this error
-                throw e;
-              }
-            }
-            pathList.add(file.getEffectivePath());
+          if (!isCopyError) {
+            // File is successfully copied, just skip this file from retry.
+            srcFile.setCopyDone(true);
+            continue;
+          }
+        }
+      } else {
+        // If destination file is missing, then retry copy
+        if (sourceFs.exists(srcPath)) {
+          // If checksum does not match, likely the file is changed/removed, retry from CM path
+          if (isSourceFileMismatch(sourceFs, srcFile)) {
+            srcFile.setIsUseSourcePath(false);
+          }
+        } else {
+          if (srcFile.isUseSourcePath()) {
+            // Source file missing, then try with CM path
+            srcFile.setIsUseSourcePath(false);
+          } else {
+            // CM path itself is missing, cannot recover from this error
+            LOG.error("File Copy Failed. Both source and CM files are missing from source. "
+                    + "Missing Source File: " + srcFile.getSourcePath() + ", CM File: " + srcFile.getCmPath() + ". "
+                    + "Try setting higher value for hive.repl.cm.retain in source warehouse. "
+                    + "Also, bootstrap the system again to get back the consistent replicated state.");
+            throw new IOException("Both source and CM path are missing from source.");
           }
         }
       }
-      repeat++;
+      srcPath = srcFile.getEffectivePath();
+      if (null == srcPath) {
+        // This case possible if CM path is not enabled.
+        LOG.error("File copy failed and likely source file is deleted or modified. "
+                + "Source File: " + srcFile.getSourcePath());
+        throw new IOException("File copy failed and likely source file is deleted or modified.");
+      }
+      pathList.add(srcPath);
     }
+    return pathList;
+  }
+
+  // Check if the source file unmodified even after copy to see if we copied the right file
+  private boolean isSourceFileMismatch(FileSystem sourceFs, ReplChangeManager.FileInfo srcFile) {
+    // If source is already CM path, the checksum will be always matching
+    if (srcFile.isUseSourcePath()) {
+      String sourceChecksumString = srcFile.getCheckSum();
+      if (sourceChecksumString != null) {
+        String verifySourceChecksumString;
+        try {
+          verifySourceChecksumString
+                  = ReplChangeManager.checksumFor(srcFile.getSourcePath(), sourceFs);
+        } catch (IOException e) {
+          // Retry with CM path
+          LOG.debug("Unable to calculate checksum for source file: " + srcFile.getSourcePath());
+          return true;
+        }
+        if (!sourceChecksumString.equals(verifySourceChecksumString)) {
+          return true;
+        }
+      }
+    }
+    return false;
   }
 
   // Copy without retry
@@ -175,7 +230,6 @@ public class CopyUtils {
       URI destinationUri = destination.toUri();
       destination = new Path(destinationUri.getScheme(), destinationUri.getAuthority(),
           RAW_RESERVED_VIRTUAL_PATH + destinationUri.getPath());
-      hiveConf.set("distcp.options.px","");
     }
 
     FileUtils.distCp(
@@ -255,16 +309,18 @@ public class CopyUtils {
       try {
         contentSummary = sourceFs.getContentSummary(fileInfo.getEffectivePath());
       } catch (IOException e) {
-        // in replication, if source file does not exist, try cmroot
+        // In replication, if source file does not exist, try cmroot
         if (fileInfo.isUseSourcePath() && fileInfo.getCmPath() != null) {
           contentSummary = sourceFs.getContentSummary(fileInfo.getCmPath());
           fileInfo.setIsUseSourcePath(false);
         }
       }
-      size += contentSummary.getLength();
-      numberOfFiles += contentSummary.getFileCount();
-      if (limitReachedForLocalCopy(size, numberOfFiles)) {
-        return false;
+      if (contentSummary != null) {
+        size += contentSummary.getLength();
+        numberOfFiles += contentSummary.getFileCount();
+        if (limitReachedForLocalCopy(size, numberOfFiles)) {
+          return false;
+        }
       }
     }
     return true;
@@ -289,7 +345,7 @@ public class CopyUtils {
     for (Path path : srcPaths) {
       FileSystem fileSystem = path.getFileSystem(hiveConf);
       if (!result.containsKey(fileSystem)) {
-        result.put(fileSystem, new ArrayList<Path>());
+        result.put(fileSystem, new ArrayList<>());
       }
       result.get(fileSystem).add(path);
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/a8fc0e67/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/DumpMetaData.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/DumpMetaData.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/DumpMetaData.java
index 9dfd7cd..974e105 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/DumpMetaData.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/DumpMetaData.java
@@ -78,7 +78,6 @@ public class DumpMetaData {
             Long.valueOf(lineContents[2]),
             new Path(lineContents[3]));
         setPayload(lineContents[4].equals(Utilities.nullStringOutput) ? null : lineContents[4]);
-        ReplChangeManager.setCmRoot(cmRoot);
       } else {
         throw new IOException(
             "Unable to read valid values from dumpFile:" + dumpFile.toUri().toString());

http://git-wip-us.apache.org/repos/asf/hive/blob/a8fc0e67/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/CreateFunctionHandler.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/CreateFunctionHandler.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/CreateFunctionHandler.java
index a3331b4..3a32885 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/CreateFunctionHandler.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/CreateFunctionHandler.java
@@ -181,14 +181,14 @@ public class CreateFunctionHandler extends AbstractMessageHandler {
     ResourceUri destinationResourceUri(ResourceUri resourceUri)
         throws IOException, SemanticException {
       String sourceUri = resourceUri.getUri();
-      String[] split = sourceUri.split(Path.SEPARATOR);
+      String[] split = ReplChangeManager.decodeFileUri(sourceUri)[0].split(Path.SEPARATOR);
       PathBuilder pathBuilder = new PathBuilder(functionsRootDir);
       Path qualifiedDestinationPath = PathBuilder.fullyQualifiedHDFSUri(
           pathBuilder
               .addDescendant(destinationDbName.toLowerCase())
               .addDescendant(metadata.function.getFunctionName().toLowerCase())
               .addDescendant(String.valueOf(System.nanoTime()))
-              .addDescendant(ReplChangeManager.getFileWithChksumFromURI(split[split.length - 1])[0])
+              .addDescendant(split[split.length - 1])
               .build(),
           new Path(functionsRootDir).getFileSystem(context.hiveConf)
       );

http://git-wip-us.apache.org/repos/asf/hive/blob/a8fc0e67/shims/0.23/src/main/java/org/apache/hadoop/hive/shims/Hadoop23Shims.java
----------------------------------------------------------------------
diff --git a/shims/0.23/src/main/java/org/apache/hadoop/hive/shims/Hadoop23Shims.java b/shims/0.23/src/main/java/org/apache/hadoop/hive/shims/Hadoop23Shims.java
index 285d616..ec06a88 100644
--- a/shims/0.23/src/main/java/org/apache/hadoop/hive/shims/Hadoop23Shims.java
+++ b/shims/0.23/src/main/java/org/apache/hadoop/hive/shims/Hadoop23Shims.java
@@ -1100,8 +1100,7 @@ public class Hadoop23Shims extends HadoopShimsSecure {
     if (params.size() == 0){
       // if no entries were added via conf, we initiate our defaults
       params.add("-update");
-      params.add("-skipcrccheck");
-      params.add("-pb");
+      params.add("-pbx");
     }
     for (Path src : srcPaths) {
       params.add(src.toString());

http://git-wip-us.apache.org/repos/asf/hive/blob/a8fc0e67/shims/0.23/src/main/test/org/apache/hadoop/hive/shims/TestHadoop23Shims.java
----------------------------------------------------------------------
diff --git a/shims/0.23/src/main/test/org/apache/hadoop/hive/shims/TestHadoop23Shims.java b/shims/0.23/src/main/test/org/apache/hadoop/hive/shims/TestHadoop23Shims.java
index 108289c..9a9311b 100644
--- a/shims/0.23/src/main/test/org/apache/hadoop/hive/shims/TestHadoop23Shims.java
+++ b/shims/0.23/src/main/test/org/apache/hadoop/hive/shims/TestHadoop23Shims.java
@@ -20,23 +20,14 @@ package org.apache.hadoop.hive.shims;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.tools.DistCpOptions;
 import org.junit.Test;
 
-import java.io.IOException;
 import java.util.Collections;
 import java.util.List;
 
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
-import static org.mockito.Matchers.any;
-import static org.mockito.Mockito.doReturn;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.spy;
-import static org.mockito.Mockito.when;
+
 
 public class TestHadoop23Shims {
 
@@ -49,12 +40,11 @@ public class TestHadoop23Shims {
     Hadoop23Shims shims = new Hadoop23Shims();
     List<String> paramsDefault = shims.constructDistCpParams(Collections.singletonList(copySrc), copyDst, conf);
 
-    assertEquals(5, paramsDefault.size());
+    assertEquals(4, paramsDefault.size());
     assertTrue("Distcp -update set by default", paramsDefault.contains("-update"));
-    assertTrue("Distcp -skipcrccheck set by default", paramsDefault.contains("-skipcrccheck"));
-    assertTrue("Distcp -pb set by default", paramsDefault.contains("-pb"));
-    assertEquals(copySrc.toString(), paramsDefault.get(3));
-    assertEquals(copyDst.toString(), paramsDefault.get(4));
+    assertTrue("Distcp -pbx set by default", paramsDefault.contains("-pbx"));
+    assertEquals(copySrc.toString(), paramsDefault.get(2));
+    assertEquals(copyDst.toString(), paramsDefault.get(3));
 
     conf.set("distcp.options.foo", "bar"); // should set "-foo bar"
     conf.set("distcp.options.blah", ""); // should set "-blah"
@@ -69,8 +59,8 @@ public class TestHadoop23Shims {
         !paramsWithCustomParamInjection.contains("-update"));
     assertTrue("Distcp -skipcrccheck not set if not requested",
         !paramsWithCustomParamInjection.contains("-skipcrccheck"));
-    assertTrue("Distcp -pb not set if not requested",
-        !paramsWithCustomParamInjection.contains("-pb"));
+    assertTrue("Distcp -pbx not set if not requested",
+        !paramsWithCustomParamInjection.contains("-pbx"));
 
     // the "-foo bar" and "-blah" params order is not guaranteed
     String firstParam = paramsWithCustomParamInjection.get(0);

http://git-wip-us.apache.org/repos/asf/hive/blob/a8fc0e67/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/ReplChangeManager.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/ReplChangeManager.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/ReplChangeManager.java
index abe1226..79ba7ff 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/ReplChangeManager.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/ReplChangeManager.java
@@ -37,8 +37,8 @@ import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
 import org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars;
 import org.apache.hadoop.hive.metastore.utils.FileUtils;
+import org.apache.hadoop.hive.metastore.utils.StringUtils;
 import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.util.StringUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -52,7 +52,6 @@ public class ReplChangeManager {
   private static Configuration conf;
   private String msUser;
   private String msGroup;
-  private FileSystem fs;
 
   private static final String ORIG_LOC_TAG = "user.original-loc";
   static final String REMAIN_IN_TRASH_TAG = "user.remain-in-trash";
@@ -70,6 +69,7 @@ public class ReplChangeManager {
     private String checkSum;
     private boolean useSourcePath;
     private String subDir;
+    private boolean copyDone;
 
     public FileInfo(FileSystem srcFs, Path sourcePath, String subDir) {
       this(srcFs, sourcePath, null, null, true, subDir);
@@ -82,6 +82,7 @@ public class ReplChangeManager {
       this.checkSum = checkSum;
       this.useSourcePath = useSourcePath;
       this.subDir = subDir;
+      this.copyDone = false;
     }
     public FileSystem getSrcFs() {
       return srcFs;
@@ -104,6 +105,12 @@ public class ReplChangeManager {
     public String getSubDir() {
       return subDir;
     }
+    public boolean isCopyDone() {
+      return copyDone;
+    }
+    public void setCopyDone(boolean copyDone) {
+      this.copyDone = copyDone;
+    }
     public Path getEffectivePath() {
       if (useSourcePath) {
         return sourcePath;
@@ -128,11 +135,11 @@ public class ReplChangeManager {
           ReplChangeManager.cmroot = new Path(MetastoreConf.getVar(conf, ConfVars.REPLCMDIR));
           ReplChangeManager.conf = conf;
 
-          fs = cmroot.getFileSystem(conf);
+          FileSystem cmFs = cmroot.getFileSystem(conf);
           // Create cmroot with permission 700 if not exist
-          if (!fs.exists(cmroot)) {
-            fs.mkdirs(cmroot);
-            fs.setPermission(cmroot, new FsPermission("700"));
+          if (!cmFs.exists(cmroot)) {
+            cmFs.mkdirs(cmroot);
+            cmFs.setPermission(cmroot, new FsPermission("700"));
           }
           UserGroupInformation usergroupInfo = UserGroupInformation.getCurrentUser();
           msUser = usergroupInfo.getShortUserName();
@@ -173,6 +180,7 @@ public class ReplChangeManager {
     }
 
     int count = 0;
+    FileSystem fs = path.getFileSystem(conf);
     if (fs.isDirectory(path)) {
       FileStatus[] files = fs.listStatus(path, hiddenFileFilter);
       for (FileStatus file : files) {
@@ -180,7 +188,7 @@ public class ReplChangeManager {
       }
     } else {
       String fileCheckSum = checksumFor(path, fs);
-      Path cmPath = getCMPath(conf, path.getName(), fileCheckSum);
+      Path cmPath = getCMPath(conf, path.getName(), fileCheckSum, cmroot.toString());
 
       // set timestamp before moving to cmroot, so we can
       // avoid race condition CM remove the file before setting
@@ -198,17 +206,15 @@ public class ReplChangeManager {
       } else {
         switch (type) {
         case MOVE: {
-          if (LOG.isDebugEnabled()) {
-            LOG.debug("Moving {} to {}", path.toString(), cmPath.toString());
-          }
+          LOG.info("Moving {} to {}", path.toString(), cmPath.toString());
+
           // Rename fails if the file with same name already exist.
           success = fs.rename(path, cmPath);
           break;
         }
         case COPY: {
-          if (LOG.isDebugEnabled()) {
-            LOG.debug("Copying {} to {}", path.toString(), cmPath.toString());
-          }
+          LOG.info("Copying {} to {}", path.toString(), cmPath.toString());
+
           // It is possible to have a file with same checksum in cmPath but the content is
           // partially copied or corrupted. In this case, just overwrite the existing file with
           // new one.
@@ -273,20 +279,17 @@ public class ReplChangeManager {
     return checksumString;
   }
 
-  static public void setCmRoot(Path cmRoot) {
-    ReplChangeManager.cmroot = cmRoot;
-  }
-
   /***
    * Convert a path of file inside a partition or table (if non-partitioned)
    *   to a deterministic location of cmroot. So user can retrieve the file back
    *   with the original location plus checksum.
-   * @param conf
+   * @param conf Hive configuration
    * @param name original filename
    * @param checkSum checksum of the file, can be retrieved by {@link #checksumFor(Path, FileSystem)}
+   * @param cmRootUri CM Root URI. (From remote source if REPL LOAD flow. From local config if recycle.)
    * @return Path
    */
-  static Path getCMPath(Configuration conf, String name, String checkSum) {
+  static Path getCMPath(Configuration conf, String name, String checkSum, String cmRootUri) {
     String newFileName = name + "_" + checkSum;
     int maxLength = conf.getInt(DFSConfigKeys.DFS_NAMENODE_MAX_COMPONENT_LENGTH_KEY,
         DFSConfigKeys.DFS_NAMENODE_MAX_COMPONENT_LENGTH_DEFAULT);
@@ -295,7 +298,7 @@ public class ReplChangeManager {
       newFileName = newFileName.substring(0, maxLength-1);
     }
 
-    return new Path(cmroot, newFileName);
+    return new Path(cmRootUri, newFileName);
   }
 
   /***
@@ -303,19 +306,20 @@ public class ReplChangeManager {
    * matches, return the file; otherwise, use chksumString to retrieve it from cmroot
    * @param src Original file location
    * @param checksumString Checksum of the original file
+   * @param srcCMRootURI CM root URI of the source cluster
    * @param subDir Sub directory to which the source file belongs to
-   * @param conf
+   * @param conf Hive configuration
    * @return Corresponding FileInfo object
    */
-  public static FileInfo getFileInfo(Path src, String checksumString, String subDir, Configuration conf)
-          throws MetaException {
+  public static FileInfo getFileInfo(Path src, String checksumString, String srcCMRootURI, String subDir,
+                                     Configuration conf) throws MetaException {
     try {
       FileSystem srcFs = src.getFileSystem(conf);
       if (checksumString == null) {
         return new FileInfo(srcFs, src, subDir);
       }
 
-      Path cmPath = getCMPath(conf, src.getName(), checksumString);
+      Path cmPath = getCMPath(conf, src.getName(), checksumString, srcCMRootURI);
       if (!srcFs.exists(src)) {
         return new FileInfo(srcFs, src, cmPath, checksumString, false, subDir);
       }
@@ -338,7 +342,7 @@ public class ReplChangeManager {
   }
 
   /***
-   * Concatenate filename, checksum  and subdirectory with "#"
+   * Concatenate filename, checksum, source cmroot uri and subdirectory with "#"
    * @param fileUriStr Filename string
    * @param fileChecksum Checksum string
    * @param encodedSubDir sub directory path into which this file belongs to. Here encoded means,
@@ -346,38 +350,50 @@ public class ReplChangeManager {
    * @return Concatenated Uri string
    */
   // TODO: this needs to be enhanced once change management based filesystem is implemented
-  // Currently using fileuri#checksum#subdirs as the format
-  public static String encodeFileUri(String fileUriStr, String fileChecksum, String encodedSubDir) {
+  // Currently using fileuri#checksum#cmrooturi#subdirs as the format
+  public static String encodeFileUri(String fileUriStr, String fileChecksum, String encodedSubDir)
+          throws IOException {
     String encodedUri = fileUriStr;
-    if (fileChecksum != null) {
-      encodedUri = encodedUri + URI_FRAGMENT_SEPARATOR + fileChecksum;
+    if ((fileChecksum != null) && (cmroot != null)) {
+      encodedUri = encodedUri + URI_FRAGMENT_SEPARATOR + fileChecksum
+              + URI_FRAGMENT_SEPARATOR + FileUtils.makeQualified(cmroot, conf);
+    } else {
+      encodedUri = encodedUri + URI_FRAGMENT_SEPARATOR + URI_FRAGMENT_SEPARATOR;
     }
-    if (encodedSubDir != null) {
-      encodedUri = encodedUri + URI_FRAGMENT_SEPARATOR + encodedSubDir;
+    encodedUri = encodedUri + URI_FRAGMENT_SEPARATOR + ((encodedSubDir != null) ? encodedSubDir : "");
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Encoded URI: " + encodedUri);
     }
     return encodedUri;
   }
 
   /***
-   * Split uri with fragment into file uri and checksum
+   * Split uri with fragment into file uri, subdirs, checksum and source cmroot uri.
+   * Currently using fileuri#checksum#cmrooturi#subdirs as the format.
    * @param fileURIStr uri with fragment
-   * @return array of file name and checksum
+   * @return array of file name, subdirs, checksum and source CM root URI
    */
-  static public String[] getFileWithChksumFromURI(String fileURIStr) {
+  public static String[] decodeFileUri(String fileURIStr) {
     String[] uriAndFragment = fileURIStr.split(URI_FRAGMENT_SEPARATOR);
-    String[] result = new String[3];
+    String[] result = new String[4];
     result[0] = uriAndFragment[0];
-    if (uriAndFragment.length>1) {
+    if ((uriAndFragment.length > 1) && !StringUtils.isEmpty(uriAndFragment[1])) {
       result[1] = uriAndFragment[1];
     }
-    if (uriAndFragment.length>2) {
+    if ((uriAndFragment.length > 2)  && !StringUtils.isEmpty(uriAndFragment[2])) {
       result[2] = uriAndFragment[2];
     }
+    if ((uriAndFragment.length > 3)  && !StringUtils.isEmpty(uriAndFragment[3])) {
+      result[3] = uriAndFragment[3];
+    }
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Reading Encoded URI: " + result[0] + ":: " + result[1] + ":: " + result[2] + ":: " + result[3]);
+    }
     return result;
   }
 
-  public static boolean isCMFileUri(Path fromPath, FileSystem srcFs) {
-    String[] result = getFileWithChksumFromURI(fromPath.toString());
+  public static boolean isCMFileUri(Path fromPath) {
+    String[] result = decodeFileUri(fromPath.toString());
     return result[1] != null;
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/a8fc0e67/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/utils/FileUtils.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/utils/FileUtils.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/utils/FileUtils.java
index 4138fa5..ec9e9e2 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/utils/FileUtils.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/utils/FileUtils.java
@@ -33,6 +33,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
+import java.net.URI;
 import java.util.ArrayList;
 import java.util.BitSet;
 import java.util.Collections;
@@ -171,9 +172,9 @@ public class FileUtils {
    * Rename a file.  Unlike {@link FileSystem#rename(Path, Path)}, if the destPath already exists
    * and is a directory, this will NOT move the sourcePath into it.  It will throw an IOException
    * instead.
-   * @param srcfs file system src paths are on
-   * @param destfs file system dest paths are on
-   * @param sourcePath source file or directory to move
+   * @param srcFs file system src paths are on
+   * @param destFs file system dest paths are on
+   * @param srcPath source file or directory to move
    * @param destPath destination file name.  This must be a file and not an existing directory.
    * @return result of fs.rename.
    * @throws IOException if fs.rename throws it, or if destPath already exists.
@@ -440,7 +441,7 @@ public class FileUtils {
    * Utility method that determines if a specified directory already has
    * contents (non-hidden files) or not - useful to determine if an
    * immutable table already has contents, for example.
-   *
+   * @param fs
    * @param path
    * @throws IOException
    */
@@ -454,4 +455,59 @@ public class FileUtils {
     }
     return true;
   }
+
+  /**
+   * Variant of Path.makeQualified that qualifies the input path against the default file system
+   * indicated by the configuration
+   *
+   * This does not require a FileSystem handle in most cases - only requires the Filesystem URI.
+   * This saves the cost of opening the Filesystem - which can involve RPCs - as well as cause
+   * errors
+   *
+   * @param path
+   *          path to be fully qualified
+   * @param conf
+   *          Configuration file
+   * @return path qualified relative to default file system
+   */
+  public static Path makeQualified(Path path, Configuration conf) throws IOException {
+
+    if (!path.isAbsolute()) {
+      // in this case we need to get the working directory
+      // and this requires a FileSystem handle. So revert to
+      // original method.
+      FileSystem fs = FileSystem.get(conf);
+      return path.makeQualified(fs.getUri(), fs.getWorkingDirectory());
+    }
+
+    URI fsUri = FileSystem.getDefaultUri(conf);
+    URI pathUri = path.toUri();
+
+    String scheme = pathUri.getScheme();
+    String authority = pathUri.getAuthority();
+
+    // validate/fill-in scheme and authority. this follows logic
+    // identical to FileSystem.get(URI, conf) - but doesn't actually
+    // obtain a file system handle
+
+    if (scheme == null) {
+      // no scheme - use default file system uri
+      scheme = fsUri.getScheme();
+      authority = fsUri.getAuthority();
+      if (authority == null) {
+        authority = "";
+      }
+    } else {
+      if (authority == null) {
+        // no authority - use default one if it applies
+        if (scheme.equals(fsUri.getScheme()) && fsUri.getAuthority() != null) {
+          authority = fsUri.getAuthority();
+        } else {
+          authority = "";
+        }
+      }
+    }
+
+    return new Path(scheme, authority, pathUri.getPath());
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/a8fc0e67/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/utils/HdfsUtils.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/utils/HdfsUtils.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/utils/HdfsUtils.java
index 6a76de5..2122788 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/utils/HdfsUtils.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/utils/HdfsUtils.java
@@ -205,8 +205,7 @@ public class HdfsUtils {
     if (params.size() == 0){
       // if no entries were added via conf, we initiate our defaults
       params.add("-update");
-      params.add("-skipcrccheck");
-      params.add("-pb");
+      params.add("-pbx");
     }
     for (Path src : srcPaths) {
       params.add(src.toString());

http://git-wip-us.apache.org/repos/asf/hive/blob/a8fc0e67/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/utils/StringUtils.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/utils/StringUtils.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/utils/StringUtils.java
index 4449799..e49a423 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/utils/StringUtils.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/utils/StringUtils.java
@@ -17,9 +17,6 @@
  */
 package org.apache.hadoop.hive.metastore.utils;
 
-import com.google.common.collect.Interner;
-import com.google.common.collect.Interners;
-
 import java.io.PrintWriter;
 import java.io.StringWriter;
 import java.util.ArrayList;
@@ -109,4 +106,25 @@ public class StringUtils {
     wrt.close();
     return stm.toString();
   }
+
+  /**
+   * Given an array of bytes it will convert the bytes to a hex string
+   * representation of the bytes.
+   * @param bytes Input bytes
+   * @param start start index, inclusively
+   * @param end end index, exclusively
+   * @return hex string representation of the byte array
+   */
+  public static String byteToHexString(byte[] bytes, int start, int end) {
+    return org.apache.hadoop.util.StringUtils.byteToHexString(bytes, start, end);
+  }
+
+  /**
+   * Checks if the input string/char sequence is empty
+   * @param cs Input char sequence
+   * @return true if empty and false if not
+   */
+  public static boolean isEmpty(CharSequence cs) {
+    return cs == null || cs.length() == 0;
+  }
 }


[48/50] [abbrv] hive git commit: Preparing for 3.0.0 release

Posted by vg...@apache.org.
Preparing for 3.0.0 release


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

Branch: refs/heads/branch-3.0.0
Commit: 04ad5d1790a5cdf91fce2b830d587e864c366bfc
Parents: 253f0ff
Author: Vineet Garg <vg...@apache.org>
Authored: Tue May 15 15:14:37 2018 -0700
Committer: Vineet Garg <vg...@apache.org>
Committed: Tue May 15 15:14:37 2018 -0700

----------------------------------------------------------------------
 .reviewboardrc                                    |  2 +-
 accumulo-handler/pom.xml                          |  2 +-
 beeline/pom.xml                                   |  2 +-
 classification/pom.xml                            |  2 +-
 cli/pom.xml                                       |  2 +-
 common/pom.xml                                    |  2 +-
 contrib/pom.xml                                   |  2 +-
 druid-handler/pom.xml                             |  2 +-
 hbase-handler/pom.xml                             |  2 +-
 hcatalog/core/pom.xml                             |  2 +-
 hcatalog/hcatalog-pig-adapter/pom.xml             |  2 +-
 hcatalog/pom.xml                                  |  2 +-
 hcatalog/server-extensions/pom.xml                |  2 +-
 hcatalog/streaming/pom.xml                        |  2 +-
 hcatalog/webhcat/java-client/pom.xml              |  2 +-
 hcatalog/webhcat/svr/pom.xml                      |  2 +-
 hplsql/pom.xml                                    |  2 +-
 itests/custom-serde/pom.xml                       |  2 +-
 itests/custom-udfs/pom.xml                        |  2 +-
 itests/custom-udfs/udf-classloader-udf1/pom.xml   |  2 +-
 itests/custom-udfs/udf-classloader-udf2/pom.xml   |  2 +-
 itests/custom-udfs/udf-classloader-util/pom.xml   |  2 +-
 .../custom-udfs/udf-vectorized-badexample/pom.xml |  2 +-
 itests/hcatalog-unit/pom.xml                      |  2 +-
 itests/hive-blobstore/pom.xml                     |  2 +-
 itests/hive-jmh/pom.xml                           |  2 +-
 itests/hive-minikdc/pom.xml                       |  2 +-
 itests/hive-unit-hadoop2/pom.xml                  |  2 +-
 itests/hive-unit/pom.xml                          |  2 +-
 itests/pom.xml                                    |  4 ++--
 itests/qtest-accumulo/pom.xml                     |  2 +-
 itests/qtest-druid/pom.xml                        |  2 +-
 itests/qtest-spark/pom.xml                        |  2 +-
 itests/qtest/pom.xml                              |  2 +-
 itests/test-serde/pom.xml                         |  2 +-
 itests/util/pom.xml                               |  2 +-
 jdbc-handler/pom.xml                              |  2 +-
 jdbc/pom.xml                                      |  2 +-
 kryo-registrator/pom.xml                          |  2 +-
 llap-client/pom.xml                               |  2 +-
 llap-common/pom.xml                               |  2 +-
 llap-ext-client/pom.xml                           |  2 +-
 llap-server/pom.xml                               |  2 +-
 llap-tez/pom.xml                                  |  2 +-
 metastore/pom.xml                                 |  2 +-
 packaging/pom.xml                                 |  2 +-
 pom.xml                                           |  5 ++---
 ql/pom.xml                                        |  2 +-
 .../apache/hadoop/hive/ql/TestTxnConcatenate.java | 18 ++++++++++++++++++
 serde/pom.xml                                     |  2 +-
 service-rpc/pom.xml                               |  2 +-
 service/pom.xml                                   |  2 +-
 shims/0.23/pom.xml                                |  2 +-
 shims/aggregator/pom.xml                          |  2 +-
 shims/common/pom.xml                              |  2 +-
 shims/pom.xml                                     |  2 +-
 shims/scheduler/pom.xml                           |  2 +-
 spark-client/pom.xml                              |  4 ++--
 standalone-metastore/pom.xml                      |  2 +-
 streaming/pom.xml                                 |  2 +-
 testutils/pom.xml                                 |  2 +-
 vector-code-gen/pom.xml                           |  2 +-
 62 files changed, 82 insertions(+), 65 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/04ad5d17/.reviewboardrc
----------------------------------------------------------------------
diff --git a/.reviewboardrc b/.reviewboardrc
index 4923167..c6a1430 100644
--- a/.reviewboardrc
+++ b/.reviewboardrc
@@ -28,6 +28,6 @@
 
 REPOSITORY='hive-git'
 REVIEWBOARD_URL='https://reviews.apache.org'
-TRACKING_BRANCH='origin/master'
+TRACKING_BRANCH='origin/branch-3'
 TARGET_GROUPS='hive'
 GUESS_FIELDS='yes'

http://git-wip-us.apache.org/repos/asf/hive/blob/04ad5d17/accumulo-handler/pom.xml
----------------------------------------------------------------------
diff --git a/accumulo-handler/pom.xml b/accumulo-handler/pom.xml
index edac1b1..c4be33e 100644
--- a/accumulo-handler/pom.xml
+++ b/accumulo-handler/pom.xml
@@ -19,7 +19,7 @@
   <parent>
     <groupId>org.apache.hive</groupId>
     <artifactId>hive</artifactId>
-    <version>3.0.0-SNAPSHOT</version>
+    <version>3.0.0</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/hive/blob/04ad5d17/beeline/pom.xml
----------------------------------------------------------------------
diff --git a/beeline/pom.xml b/beeline/pom.xml
index b0a9a0b..e42f944b 100644
--- a/beeline/pom.xml
+++ b/beeline/pom.xml
@@ -19,7 +19,7 @@
   <parent>
     <groupId>org.apache.hive</groupId>
     <artifactId>hive</artifactId>
-    <version>3.0.0-SNAPSHOT</version>
+    <version>3.0.0</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/hive/blob/04ad5d17/classification/pom.xml
----------------------------------------------------------------------
diff --git a/classification/pom.xml b/classification/pom.xml
index ff4b795..1dbd7f0 100644
--- a/classification/pom.xml
+++ b/classification/pom.xml
@@ -19,7 +19,7 @@
   <parent>
     <groupId>org.apache.hive</groupId>
     <artifactId>hive</artifactId>
-    <version>3.0.0-SNAPSHOT</version>
+    <version>3.0.0</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/hive/blob/04ad5d17/cli/pom.xml
----------------------------------------------------------------------
diff --git a/cli/pom.xml b/cli/pom.xml
index 71d214b..9c9834f 100644
--- a/cli/pom.xml
+++ b/cli/pom.xml
@@ -19,7 +19,7 @@
   <parent>
     <groupId>org.apache.hive</groupId>
     <artifactId>hive</artifactId>
-    <version>3.0.0-SNAPSHOT</version>
+    <version>3.0.0</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/hive/blob/04ad5d17/common/pom.xml
----------------------------------------------------------------------
diff --git a/common/pom.xml b/common/pom.xml
index 9393d87..8deb67f 100644
--- a/common/pom.xml
+++ b/common/pom.xml
@@ -19,7 +19,7 @@
   <parent>
     <groupId>org.apache.hive</groupId>
     <artifactId>hive</artifactId>
-    <version>3.0.0-SNAPSHOT</version>
+    <version>3.0.0</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/hive/blob/04ad5d17/contrib/pom.xml
----------------------------------------------------------------------
diff --git a/contrib/pom.xml b/contrib/pom.xml
index 7423e31..56dd688 100644
--- a/contrib/pom.xml
+++ b/contrib/pom.xml
@@ -19,7 +19,7 @@
   <parent>
     <groupId>org.apache.hive</groupId>
     <artifactId>hive</artifactId>
-    <version>3.0.0-SNAPSHOT</version>
+    <version>3.0.0</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/hive/blob/04ad5d17/druid-handler/pom.xml
----------------------------------------------------------------------
diff --git a/druid-handler/pom.xml b/druid-handler/pom.xml
index 9c2bebf..ee81edf 100644
--- a/druid-handler/pom.xml
+++ b/druid-handler/pom.xml
@@ -19,7 +19,7 @@
   <parent>
     <groupId>org.apache.hive</groupId>
     <artifactId>hive</artifactId>
-    <version>3.0.0-SNAPSHOT</version>
+    <version>3.0.0</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/hive/blob/04ad5d17/hbase-handler/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-handler/pom.xml b/hbase-handler/pom.xml
index 59f9dd8..b3d977f 100644
--- a/hbase-handler/pom.xml
+++ b/hbase-handler/pom.xml
@@ -19,7 +19,7 @@
   <parent>
     <groupId>org.apache.hive</groupId>
     <artifactId>hive</artifactId>
-    <version>3.0.0-SNAPSHOT</version>
+    <version>3.0.0</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/hive/blob/04ad5d17/hcatalog/core/pom.xml
----------------------------------------------------------------------
diff --git a/hcatalog/core/pom.xml b/hcatalog/core/pom.xml
index c6831ea..ddb6d16 100644
--- a/hcatalog/core/pom.xml
+++ b/hcatalog/core/pom.xml
@@ -25,7 +25,7 @@
   <parent>
     <groupId>org.apache.hive.hcatalog</groupId>
     <artifactId>hive-hcatalog</artifactId>
-    <version>3.0.0-SNAPSHOT</version>
+    <version>3.0.0</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/hive/blob/04ad5d17/hcatalog/hcatalog-pig-adapter/pom.xml
----------------------------------------------------------------------
diff --git a/hcatalog/hcatalog-pig-adapter/pom.xml b/hcatalog/hcatalog-pig-adapter/pom.xml
index 4a2075f..77234c2 100644
--- a/hcatalog/hcatalog-pig-adapter/pom.xml
+++ b/hcatalog/hcatalog-pig-adapter/pom.xml
@@ -25,7 +25,7 @@
   <parent>
     <groupId>org.apache.hive.hcatalog</groupId>
     <artifactId>hive-hcatalog</artifactId>
-    <version>3.0.0-SNAPSHOT</version>
+    <version>3.0.0</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/hive/blob/04ad5d17/hcatalog/pom.xml
----------------------------------------------------------------------
diff --git a/hcatalog/pom.xml b/hcatalog/pom.xml
index 9a73c84..7834a68 100644
--- a/hcatalog/pom.xml
+++ b/hcatalog/pom.xml
@@ -24,7 +24,7 @@
   <parent>
     <groupId>org.apache.hive</groupId>
     <artifactId>hive</artifactId>
-    <version>3.0.0-SNAPSHOT</version>
+    <version>3.0.0</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/hive/blob/04ad5d17/hcatalog/server-extensions/pom.xml
----------------------------------------------------------------------
diff --git a/hcatalog/server-extensions/pom.xml b/hcatalog/server-extensions/pom.xml
index c8096e8..af63948 100644
--- a/hcatalog/server-extensions/pom.xml
+++ b/hcatalog/server-extensions/pom.xml
@@ -25,7 +25,7 @@
   <parent>
     <groupId>org.apache.hive.hcatalog</groupId>
     <artifactId>hive-hcatalog</artifactId>
-    <version>3.0.0-SNAPSHOT</version>
+    <version>3.0.0</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/hive/blob/04ad5d17/hcatalog/streaming/pom.xml
----------------------------------------------------------------------
diff --git a/hcatalog/streaming/pom.xml b/hcatalog/streaming/pom.xml
index 5bea0a6..686265a 100644
--- a/hcatalog/streaming/pom.xml
+++ b/hcatalog/streaming/pom.xml
@@ -20,7 +20,7 @@
   <parent>
     <groupId>org.apache.hive.hcatalog</groupId>
     <artifactId>hive-hcatalog</artifactId>
-    <version>3.0.0-SNAPSHOT</version>
+    <version>3.0.0</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/hive/blob/04ad5d17/hcatalog/webhcat/java-client/pom.xml
----------------------------------------------------------------------
diff --git a/hcatalog/webhcat/java-client/pom.xml b/hcatalog/webhcat/java-client/pom.xml
index 5693157..0686b36 100644
--- a/hcatalog/webhcat/java-client/pom.xml
+++ b/hcatalog/webhcat/java-client/pom.xml
@@ -25,7 +25,7 @@
   <parent>
     <groupId>org.apache.hive.hcatalog</groupId>
     <artifactId>hive-hcatalog</artifactId>
-    <version>3.0.0-SNAPSHOT</version>
+    <version>3.0.0</version>
     <relativePath>../../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/hive/blob/04ad5d17/hcatalog/webhcat/svr/pom.xml
----------------------------------------------------------------------
diff --git a/hcatalog/webhcat/svr/pom.xml b/hcatalog/webhcat/svr/pom.xml
index 615f9c7..eaf1b24 100644
--- a/hcatalog/webhcat/svr/pom.xml
+++ b/hcatalog/webhcat/svr/pom.xml
@@ -25,7 +25,7 @@
   <parent>
     <groupId>org.apache.hive.hcatalog</groupId>
     <artifactId>hive-hcatalog</artifactId>
-    <version>3.0.0-SNAPSHOT</version>
+    <version>3.0.0</version>
     <relativePath>../../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/hive/blob/04ad5d17/hplsql/pom.xml
----------------------------------------------------------------------
diff --git a/hplsql/pom.xml b/hplsql/pom.xml
index 44da8b2..de9b20f 100644
--- a/hplsql/pom.xml
+++ b/hplsql/pom.xml
@@ -19,7 +19,7 @@
   <parent>
     <groupId>org.apache.hive</groupId>
     <artifactId>hive</artifactId>
-    <version>3.0.0-SNAPSHOT</version>
+    <version>3.0.0</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/hive/blob/04ad5d17/itests/custom-serde/pom.xml
----------------------------------------------------------------------
diff --git a/itests/custom-serde/pom.xml b/itests/custom-serde/pom.xml
index 78b68c5..94e67d4 100644
--- a/itests/custom-serde/pom.xml
+++ b/itests/custom-serde/pom.xml
@@ -19,7 +19,7 @@
   <parent>
     <groupId>org.apache.hive</groupId>
     <artifactId>hive-it</artifactId>
-    <version>3.0.0-SNAPSHOT</version>
+    <version>3.0.0</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/hive/blob/04ad5d17/itests/custom-udfs/pom.xml
----------------------------------------------------------------------
diff --git a/itests/custom-udfs/pom.xml b/itests/custom-udfs/pom.xml
index de7df16..36e3f2d 100644
--- a/itests/custom-udfs/pom.xml
+++ b/itests/custom-udfs/pom.xml
@@ -19,7 +19,7 @@ limitations under the License.
   <parent>
     <groupId>org.apache.hive</groupId>
     <artifactId>hive-it</artifactId>
-    <version>3.0.0-SNAPSHOT</version>
+    <version>3.0.0</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/hive/blob/04ad5d17/itests/custom-udfs/udf-classloader-udf1/pom.xml
----------------------------------------------------------------------
diff --git a/itests/custom-udfs/udf-classloader-udf1/pom.xml b/itests/custom-udfs/udf-classloader-udf1/pom.xml
index f863efd..e089758 100644
--- a/itests/custom-udfs/udf-classloader-udf1/pom.xml
+++ b/itests/custom-udfs/udf-classloader-udf1/pom.xml
@@ -19,7 +19,7 @@
   <parent>
     <groupId>org.apache.hive</groupId>
     <artifactId>hive-it-custom-udfs</artifactId>
-    <version>3.0.0-SNAPSHOT</version>
+    <version>3.0.0</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/hive/blob/04ad5d17/itests/custom-udfs/udf-classloader-udf2/pom.xml
----------------------------------------------------------------------
diff --git a/itests/custom-udfs/udf-classloader-udf2/pom.xml b/itests/custom-udfs/udf-classloader-udf2/pom.xml
index 2553f3e..24403e3 100644
--- a/itests/custom-udfs/udf-classloader-udf2/pom.xml
+++ b/itests/custom-udfs/udf-classloader-udf2/pom.xml
@@ -19,7 +19,7 @@
   <parent>
     <groupId>org.apache.hive</groupId>
     <artifactId>hive-it-custom-udfs</artifactId>
-    <version>3.0.0-SNAPSHOT</version>
+    <version>3.0.0</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/hive/blob/04ad5d17/itests/custom-udfs/udf-classloader-util/pom.xml
----------------------------------------------------------------------
diff --git a/itests/custom-udfs/udf-classloader-util/pom.xml b/itests/custom-udfs/udf-classloader-util/pom.xml
index 565a661..63d3029 100644
--- a/itests/custom-udfs/udf-classloader-util/pom.xml
+++ b/itests/custom-udfs/udf-classloader-util/pom.xml
@@ -19,7 +19,7 @@
   <parent>
     <groupId>org.apache.hive</groupId>
     <artifactId>hive-it-custom-udfs</artifactId>
-    <version>3.0.0-SNAPSHOT</version>
+    <version>3.0.0</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/hive/blob/04ad5d17/itests/custom-udfs/udf-vectorized-badexample/pom.xml
----------------------------------------------------------------------
diff --git a/itests/custom-udfs/udf-vectorized-badexample/pom.xml b/itests/custom-udfs/udf-vectorized-badexample/pom.xml
index 6dc923d..d37e54c 100644
--- a/itests/custom-udfs/udf-vectorized-badexample/pom.xml
+++ b/itests/custom-udfs/udf-vectorized-badexample/pom.xml
@@ -19,7 +19,7 @@
   <parent>
     <groupId>org.apache.hive</groupId>
     <artifactId>hive-it-custom-udfs</artifactId>
-    <version>3.0.0-SNAPSHOT</version>
+    <version>3.0.0</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/hive/blob/04ad5d17/itests/hcatalog-unit/pom.xml
----------------------------------------------------------------------
diff --git a/itests/hcatalog-unit/pom.xml b/itests/hcatalog-unit/pom.xml
index 433147a..a4c1b16 100644
--- a/itests/hcatalog-unit/pom.xml
+++ b/itests/hcatalog-unit/pom.xml
@@ -25,7 +25,7 @@
   <parent>
     <groupId>org.apache.hive</groupId>
     <artifactId>hive-it</artifactId>
-    <version>3.0.0-SNAPSHOT</version>
+    <version>3.0.0</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/hive/blob/04ad5d17/itests/hive-blobstore/pom.xml
----------------------------------------------------------------------
diff --git a/itests/hive-blobstore/pom.xml b/itests/hive-blobstore/pom.xml
index 2f45327..834bc20 100644
--- a/itests/hive-blobstore/pom.xml
+++ b/itests/hive-blobstore/pom.xml
@@ -20,7 +20,7 @@
   <parent>
     <groupId>org.apache.hive</groupId>
     <artifactId>hive-it</artifactId>
-    <version>3.0.0-SNAPSHOT</version>
+    <version>3.0.0</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/hive/blob/04ad5d17/itests/hive-jmh/pom.xml
----------------------------------------------------------------------
diff --git a/itests/hive-jmh/pom.xml b/itests/hive-jmh/pom.xml
index 94abbbf..cb12e50 100644
--- a/itests/hive-jmh/pom.xml
+++ b/itests/hive-jmh/pom.xml
@@ -19,7 +19,7 @@
   <parent>
     <groupId>org.apache.hive</groupId>
     <artifactId>hive-it</artifactId>
-    <version>3.0.0-SNAPSHOT</version>
+    <version>3.0.0</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/hive/blob/04ad5d17/itests/hive-minikdc/pom.xml
----------------------------------------------------------------------
diff --git a/itests/hive-minikdc/pom.xml b/itests/hive-minikdc/pom.xml
index 1e40d9d..4d705b2 100644
--- a/itests/hive-minikdc/pom.xml
+++ b/itests/hive-minikdc/pom.xml
@@ -19,7 +19,7 @@
   <parent>
     <groupId>org.apache.hive</groupId>
     <artifactId>hive-it</artifactId>
-    <version>3.0.0-SNAPSHOT</version>
+    <version>3.0.0</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/hive/blob/04ad5d17/itests/hive-unit-hadoop2/pom.xml
----------------------------------------------------------------------
diff --git a/itests/hive-unit-hadoop2/pom.xml b/itests/hive-unit-hadoop2/pom.xml
index 85a6145..4dd0e05 100644
--- a/itests/hive-unit-hadoop2/pom.xml
+++ b/itests/hive-unit-hadoop2/pom.xml
@@ -19,7 +19,7 @@
   <parent>
     <groupId>org.apache.hive</groupId>
     <artifactId>hive-it</artifactId>
-    <version>3.0.0-SNAPSHOT</version>
+    <version>3.0.0</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/hive/blob/04ad5d17/itests/hive-unit/pom.xml
----------------------------------------------------------------------
diff --git a/itests/hive-unit/pom.xml b/itests/hive-unit/pom.xml
index eb57c53..c52ba68 100644
--- a/itests/hive-unit/pom.xml
+++ b/itests/hive-unit/pom.xml
@@ -19,7 +19,7 @@
   <parent>
     <groupId>org.apache.hive</groupId>
     <artifactId>hive-it</artifactId>
-    <version>3.0.0-SNAPSHOT</version>
+    <version>3.0.0</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/hive/blob/04ad5d17/itests/pom.xml
----------------------------------------------------------------------
diff --git a/itests/pom.xml b/itests/pom.xml
index 40bb054..ce26178 100644
--- a/itests/pom.xml
+++ b/itests/pom.xml
@@ -19,14 +19,14 @@
   <parent>
     <groupId>org.apache.hive</groupId>
     <artifactId>hive</artifactId>
-    <version>3.0.0-SNAPSHOT</version>
+    <version>3.0.0</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 
   <groupId>org.apache.hive</groupId>
   <artifactId>hive-it</artifactId>
   <packaging>pom</packaging>
-  <version>3.0.0-SNAPSHOT</version>
+  <version>3.0.0</version>
   <name>Hive Integration - Parent</name>
 
   <properties>

http://git-wip-us.apache.org/repos/asf/hive/blob/04ad5d17/itests/qtest-accumulo/pom.xml
----------------------------------------------------------------------
diff --git a/itests/qtest-accumulo/pom.xml b/itests/qtest-accumulo/pom.xml
index 1386ce7..f3218ee 100644
--- a/itests/qtest-accumulo/pom.xml
+++ b/itests/qtest-accumulo/pom.xml
@@ -20,7 +20,7 @@
   <parent>
     <groupId>org.apache.hive</groupId>
     <artifactId>hive-it</artifactId>
-    <version>3.0.0-SNAPSHOT</version>
+    <version>3.0.0</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/hive/blob/04ad5d17/itests/qtest-druid/pom.xml
----------------------------------------------------------------------
diff --git a/itests/qtest-druid/pom.xml b/itests/qtest-druid/pom.xml
index 2e19ce5..fc660b8 100644
--- a/itests/qtest-druid/pom.xml
+++ b/itests/qtest-druid/pom.xml
@@ -23,7 +23,7 @@
   <parent>
     <artifactId>hive-it</artifactId>
     <groupId>org.apache.hive</groupId>
-    <version>3.0.0-SNAPSHOT</version>
+    <version>3.0.0</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
   <modelVersion>4.0.0</modelVersion>

http://git-wip-us.apache.org/repos/asf/hive/blob/04ad5d17/itests/qtest-spark/pom.xml
----------------------------------------------------------------------
diff --git a/itests/qtest-spark/pom.xml b/itests/qtest-spark/pom.xml
index 72b13a1..d1e65fc 100644
--- a/itests/qtest-spark/pom.xml
+++ b/itests/qtest-spark/pom.xml
@@ -20,7 +20,7 @@
   <parent>
     <groupId>org.apache.hive</groupId>
     <artifactId>hive-it</artifactId>
-    <version>3.0.0-SNAPSHOT</version>
+    <version>3.0.0</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/hive/blob/04ad5d17/itests/qtest/pom.xml
----------------------------------------------------------------------
diff --git a/itests/qtest/pom.xml b/itests/qtest/pom.xml
index f60ae1b..ac0d847 100644
--- a/itests/qtest/pom.xml
+++ b/itests/qtest/pom.xml
@@ -20,7 +20,7 @@
   <parent>
     <groupId>org.apache.hive</groupId>
     <artifactId>hive-it</artifactId>
-    <version>3.0.0-SNAPSHOT</version>
+    <version>3.0.0</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/hive/blob/04ad5d17/itests/test-serde/pom.xml
----------------------------------------------------------------------
diff --git a/itests/test-serde/pom.xml b/itests/test-serde/pom.xml
index bf5f5d2..85d3b36 100644
--- a/itests/test-serde/pom.xml
+++ b/itests/test-serde/pom.xml
@@ -19,7 +19,7 @@
   <parent>
     <groupId>org.apache.hive</groupId>
     <artifactId>hive-it</artifactId>
-    <version>3.0.0-SNAPSHOT</version>
+    <version>3.0.0</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/hive/blob/04ad5d17/itests/util/pom.xml
----------------------------------------------------------------------
diff --git a/itests/util/pom.xml b/itests/util/pom.xml
index abff200..e5bcdf6 100644
--- a/itests/util/pom.xml
+++ b/itests/util/pom.xml
@@ -19,7 +19,7 @@
   <parent>
     <groupId>org.apache.hive</groupId>
     <artifactId>hive-it</artifactId>
-    <version>3.0.0-SNAPSHOT</version>
+    <version>3.0.0</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/hive/blob/04ad5d17/jdbc-handler/pom.xml
----------------------------------------------------------------------
diff --git a/jdbc-handler/pom.xml b/jdbc-handler/pom.xml
index 6c6e1fa..c7fb04c 100644
--- a/jdbc-handler/pom.xml
+++ b/jdbc-handler/pom.xml
@@ -19,7 +19,7 @@
   <parent>
     <groupId>org.apache.hive</groupId>
     <artifactId>hive</artifactId>
-    <version>3.0.0-SNAPSHOT</version>
+    <version>3.0.0</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/hive/blob/04ad5d17/jdbc/pom.xml
----------------------------------------------------------------------
diff --git a/jdbc/pom.xml b/jdbc/pom.xml
index 8710a8b..fa9b2ba 100644
--- a/jdbc/pom.xml
+++ b/jdbc/pom.xml
@@ -19,7 +19,7 @@
   <parent>
     <groupId>org.apache.hive</groupId>
     <artifactId>hive</artifactId>
-    <version>3.0.0-SNAPSHOT</version>
+    <version>3.0.0</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/hive/blob/04ad5d17/kryo-registrator/pom.xml
----------------------------------------------------------------------
diff --git a/kryo-registrator/pom.xml b/kryo-registrator/pom.xml
index a5279fa..0ac4b35 100644
--- a/kryo-registrator/pom.xml
+++ b/kryo-registrator/pom.xml
@@ -20,7 +20,7 @@
   <parent>
     <artifactId>hive</artifactId>
     <groupId>org.apache.hive</groupId>
-    <version>3.0.0-SNAPSHOT</version>
+    <version>3.0.0</version>
   </parent>
 
   <artifactId>hive-kryo-registrator</artifactId>

http://git-wip-us.apache.org/repos/asf/hive/blob/04ad5d17/llap-client/pom.xml
----------------------------------------------------------------------
diff --git a/llap-client/pom.xml b/llap-client/pom.xml
index aa2cf32..7e7d4de 100644
--- a/llap-client/pom.xml
+++ b/llap-client/pom.xml
@@ -19,7 +19,7 @@
   <parent>
     <groupId>org.apache.hive</groupId>
     <artifactId>hive</artifactId>
-    <version>3.0.0-SNAPSHOT</version>
+    <version>3.0.0</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/hive/blob/04ad5d17/llap-common/pom.xml
----------------------------------------------------------------------
diff --git a/llap-common/pom.xml b/llap-common/pom.xml
index a4771c2..2d3439a 100644
--- a/llap-common/pom.xml
+++ b/llap-common/pom.xml
@@ -19,7 +19,7 @@
   <parent>
     <groupId>org.apache.hive</groupId>
     <artifactId>hive</artifactId>
-    <version>3.0.0-SNAPSHOT</version>
+    <version>3.0.0</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/hive/blob/04ad5d17/llap-ext-client/pom.xml
----------------------------------------------------------------------
diff --git a/llap-ext-client/pom.xml b/llap-ext-client/pom.xml
index d9ea026..a36b173 100644
--- a/llap-ext-client/pom.xml
+++ b/llap-ext-client/pom.xml
@@ -19,7 +19,7 @@
   <parent>
     <groupId>org.apache.hive</groupId>
     <artifactId>hive</artifactId>
-    <version>3.0.0-SNAPSHOT</version>
+    <version>3.0.0</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/hive/blob/04ad5d17/llap-server/pom.xml
----------------------------------------------------------------------
diff --git a/llap-server/pom.xml b/llap-server/pom.xml
index 65cb78f..1522958 100644
--- a/llap-server/pom.xml
+++ b/llap-server/pom.xml
@@ -19,7 +19,7 @@
   <parent>
     <groupId>org.apache.hive</groupId>
     <artifactId>hive</artifactId>
-    <version>3.0.0-SNAPSHOT</version>
+    <version>3.0.0</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/hive/blob/04ad5d17/llap-tez/pom.xml
----------------------------------------------------------------------
diff --git a/llap-tez/pom.xml b/llap-tez/pom.xml
index 50865ad..aee0ab0 100644
--- a/llap-tez/pom.xml
+++ b/llap-tez/pom.xml
@@ -19,7 +19,7 @@
   <parent>
     <groupId>org.apache.hive</groupId>
     <artifactId>hive</artifactId>
-    <version>3.0.0-SNAPSHOT</version>
+    <version>3.0.0</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/hive/blob/04ad5d17/metastore/pom.xml
----------------------------------------------------------------------
diff --git a/metastore/pom.xml b/metastore/pom.xml
index 21c607f..f9f3b37 100644
--- a/metastore/pom.xml
+++ b/metastore/pom.xml
@@ -19,7 +19,7 @@
   <parent>
     <groupId>org.apache.hive</groupId>
     <artifactId>hive</artifactId>
-    <version>3.0.0-SNAPSHOT</version>
+    <version>3.0.0</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/hive/blob/04ad5d17/packaging/pom.xml
----------------------------------------------------------------------
diff --git a/packaging/pom.xml b/packaging/pom.xml
index 8b03e2e..c4756b1 100644
--- a/packaging/pom.xml
+++ b/packaging/pom.xml
@@ -19,7 +19,7 @@
   <parent>
     <groupId>org.apache.hive</groupId>
     <artifactId>hive</artifactId>
-    <version>3.0.0-SNAPSHOT</version>
+    <version>3.0.0</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/hive/blob/04ad5d17/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 357a8d9..c57ff58 100644
--- a/pom.xml
+++ b/pom.xml
@@ -21,7 +21,7 @@
   </parent>
   <groupId>org.apache.hive</groupId>
   <artifactId>hive</artifactId>
-  <version>3.0.0-SNAPSHOT</version>
+  <version>3.0.0</version>
   <packaging>pom</packaging>
 
   <name>Hive</name>
@@ -31,7 +31,6 @@
   </prerequisites>
 
   <modules>
-    <module>storage-api</module>
     <module>accumulo-handler</module>
     <module>vector-code-gen</module>
     <module>beeline</module>
@@ -196,7 +195,7 @@
     <stax.version>1.0.1</stax.version>
     <slf4j.version>1.7.10</slf4j.version>
     <ST4.version>4.0.4</ST4.version>
-    <storage-api.version>2.6.0-SNAPSHOT</storage-api.version>
+    <storage-api.version>2.6.1</storage-api.version>
     <tez.version>0.9.1</tez.version>
     <slider.version>0.92.0-incubating</slider.version>
     <super-csv.version>2.2.0</super-csv.version>

http://git-wip-us.apache.org/repos/asf/hive/blob/04ad5d17/ql/pom.xml
----------------------------------------------------------------------
diff --git a/ql/pom.xml b/ql/pom.xml
index 46ce96a..ac8f641 100644
--- a/ql/pom.xml
+++ b/ql/pom.xml
@@ -19,7 +19,7 @@
   <parent>
     <groupId>org.apache.hive</groupId>
     <artifactId>hive</artifactId>
-    <version>3.0.0-SNAPSHOT</version>
+    <version>3.0.0</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/hive/blob/04ad5d17/ql/src/test/org/apache/hadoop/hive/ql/TestTxnConcatenate.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/TestTxnConcatenate.java b/ql/src/test/org/apache/hadoop/hive/ql/TestTxnConcatenate.java
index 92bcefe..2663fec 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/TestTxnConcatenate.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/TestTxnConcatenate.java
@@ -1,3 +1,21 @@
+/*
+ * 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;
 
 import org.apache.hadoop.hive.conf.HiveConf;

http://git-wip-us.apache.org/repos/asf/hive/blob/04ad5d17/serde/pom.xml
----------------------------------------------------------------------
diff --git a/serde/pom.xml b/serde/pom.xml
index fe5d2d6..884b1a3 100644
--- a/serde/pom.xml
+++ b/serde/pom.xml
@@ -19,7 +19,7 @@
   <parent>
     <groupId>org.apache.hive</groupId>
     <artifactId>hive</artifactId>
-    <version>3.0.0-SNAPSHOT</version>
+    <version>3.0.0</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/hive/blob/04ad5d17/service-rpc/pom.xml
----------------------------------------------------------------------
diff --git a/service-rpc/pom.xml b/service-rpc/pom.xml
index f338d4f..71e283c 100644
--- a/service-rpc/pom.xml
+++ b/service-rpc/pom.xml
@@ -19,7 +19,7 @@
   <parent>
     <groupId>org.apache.hive</groupId>
     <artifactId>hive</artifactId>
-    <version>3.0.0-SNAPSHOT</version>
+    <version>3.0.0</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/hive/blob/04ad5d17/service/pom.xml
----------------------------------------------------------------------
diff --git a/service/pom.xml b/service/pom.xml
index e3774df..0c03bd9 100644
--- a/service/pom.xml
+++ b/service/pom.xml
@@ -19,7 +19,7 @@
   <parent>
     <groupId>org.apache.hive</groupId>
     <artifactId>hive</artifactId>
-    <version>3.0.0-SNAPSHOT</version>
+    <version>3.0.0</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/hive/blob/04ad5d17/shims/0.23/pom.xml
----------------------------------------------------------------------
diff --git a/shims/0.23/pom.xml b/shims/0.23/pom.xml
index 61cf459..7e760fe 100644
--- a/shims/0.23/pom.xml
+++ b/shims/0.23/pom.xml
@@ -19,7 +19,7 @@
   <parent>
     <groupId>org.apache.hive</groupId>
     <artifactId>hive</artifactId>
-    <version>3.0.0-SNAPSHOT</version>
+    <version>3.0.0</version>
     <relativePath>../../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/hive/blob/04ad5d17/shims/aggregator/pom.xml
----------------------------------------------------------------------
diff --git a/shims/aggregator/pom.xml b/shims/aggregator/pom.xml
index 34f1cb0..63d851e 100644
--- a/shims/aggregator/pom.xml
+++ b/shims/aggregator/pom.xml
@@ -19,7 +19,7 @@
   <parent>
     <groupId>org.apache.hive</groupId>
     <artifactId>hive</artifactId>
-    <version>3.0.0-SNAPSHOT</version>
+    <version>3.0.0</version>
     <relativePath>../../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/hive/blob/04ad5d17/shims/common/pom.xml
----------------------------------------------------------------------
diff --git a/shims/common/pom.xml b/shims/common/pom.xml
index 209271b..e3c7899 100644
--- a/shims/common/pom.xml
+++ b/shims/common/pom.xml
@@ -19,7 +19,7 @@
   <parent>
     <groupId>org.apache.hive</groupId>
     <artifactId>hive</artifactId>
-    <version>3.0.0-SNAPSHOT</version>
+    <version>3.0.0</version>
     <relativePath>../../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/hive/blob/04ad5d17/shims/pom.xml
----------------------------------------------------------------------
diff --git a/shims/pom.xml b/shims/pom.xml
index ab3c390..dc29ae1 100644
--- a/shims/pom.xml
+++ b/shims/pom.xml
@@ -19,7 +19,7 @@
   <parent>
     <groupId>org.apache.hive</groupId>
     <artifactId>hive</artifactId>
-    <version>3.0.0-SNAPSHOT</version>
+    <version>3.0.0</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/hive/blob/04ad5d17/shims/scheduler/pom.xml
----------------------------------------------------------------------
diff --git a/shims/scheduler/pom.xml b/shims/scheduler/pom.xml
index 076e2ee..a3f3814 100644
--- a/shims/scheduler/pom.xml
+++ b/shims/scheduler/pom.xml
@@ -19,7 +19,7 @@
   <parent>
     <groupId>org.apache.hive</groupId>
     <artifactId>hive</artifactId>
-    <version>3.0.0-SNAPSHOT</version>
+    <version>3.0.0</version>
     <relativePath>../../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/hive/blob/04ad5d17/spark-client/pom.xml
----------------------------------------------------------------------
diff --git a/spark-client/pom.xml b/spark-client/pom.xml
index 6d95ae4..594518f 100644
--- a/spark-client/pom.xml
+++ b/spark-client/pom.xml
@@ -22,14 +22,14 @@
   <parent>
     <groupId>org.apache.hive</groupId>
     <artifactId>hive</artifactId>
-    <version>3.0.0-SNAPSHOT</version>
+    <version>3.0.0</version>
   </parent>
 
   <groupId>org.apache.hive</groupId>
   <artifactId>hive-spark-client</artifactId>
   <packaging>jar</packaging>
   <name>Hive Spark Remote Client</name>
-  <version>3.0.0-SNAPSHOT</version>
+  <version>3.0.0</version>
 
   <properties>
     <hive.path.to.root>..</hive.path.to.root>

http://git-wip-us.apache.org/repos/asf/hive/blob/04ad5d17/standalone-metastore/pom.xml
----------------------------------------------------------------------
diff --git a/standalone-metastore/pom.xml b/standalone-metastore/pom.xml
index 57bd49a..4095379 100644
--- a/standalone-metastore/pom.xml
+++ b/standalone-metastore/pom.xml
@@ -25,7 +25,7 @@
 
   <groupId>org.apache.hive</groupId>
   <artifactId>hive-standalone-metastore</artifactId>
-  <version>3.0.0-SNAPSHOT</version>
+  <version>3.0.0</version>
   <packaging>jar</packaging>
   <name>Hive Standalone Metastore</name>
 

http://git-wip-us.apache.org/repos/asf/hive/blob/04ad5d17/streaming/pom.xml
----------------------------------------------------------------------
diff --git a/streaming/pom.xml b/streaming/pom.xml
index ca92a29..f64c926 100644
--- a/streaming/pom.xml
+++ b/streaming/pom.xml
@@ -20,7 +20,7 @@
   <parent>
     <groupId>org.apache.hive</groupId>
     <artifactId>hive</artifactId>
-    <version>3.0.0-SNAPSHOT</version>
+    <version>3.0.0</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/hive/blob/04ad5d17/testutils/pom.xml
----------------------------------------------------------------------
diff --git a/testutils/pom.xml b/testutils/pom.xml
index 20bc122..5a093fa 100644
--- a/testutils/pom.xml
+++ b/testutils/pom.xml
@@ -19,7 +19,7 @@
   <parent>
     <groupId>org.apache.hive</groupId>
     <artifactId>hive</artifactId>
-    <version>3.0.0-SNAPSHOT</version>
+    <version>3.0.0</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/hive/blob/04ad5d17/vector-code-gen/pom.xml
----------------------------------------------------------------------
diff --git a/vector-code-gen/pom.xml b/vector-code-gen/pom.xml
index 8407de3..e7bc75e 100644
--- a/vector-code-gen/pom.xml
+++ b/vector-code-gen/pom.xml
@@ -19,7 +19,7 @@
   <parent>
     <groupId>org.apache.hive</groupId>
     <artifactId>hive</artifactId>
-    <version>3.0.0-SNAPSHOT</version>
+    <version>3.0.0</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 


[43/50] [abbrv] hive git commit: HIVE-19494: Accept shade prefix during reflective instantiation of output format (Prasanth Jayachandran reviewed by Sergey Shelukhin)

Posted by vg...@apache.org.
HIVE-19494: Accept shade prefix during reflective instantiation of output format (Prasanth Jayachandran reviewed by Sergey Shelukhin)


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

Branch: refs/heads/branch-3.0.0
Commit: 66f6748a069aca27532b5e99721b6230145886db
Parents: c740e32
Author: Prasanth Jayachandran <pr...@apache.org>
Authored: Mon May 14 16:11:18 2018 -0700
Committer: Prasanth Jayachandran <pr...@apache.org>
Committed: Mon May 14 16:11:18 2018 -0700

----------------------------------------------------------------------
 .../apache/hadoop/hive/common/JavaUtils.java    |  4 ++
 .../org/apache/hadoop/hive/conf/HiveConf.java   |  3 ++
 .../hive/streaming/AbstractRecordWriter.java    | 50 ++++++++++++--------
 3 files changed, 37 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/66f6748a/common/src/java/org/apache/hadoop/hive/common/JavaUtils.java
----------------------------------------------------------------------
diff --git a/common/src/java/org/apache/hadoop/hive/common/JavaUtils.java b/common/src/java/org/apache/hadoop/hive/common/JavaUtils.java
index e09dec1..c011cd1 100644
--- a/common/src/java/org/apache/hadoop/hive/common/JavaUtils.java
+++ b/common/src/java/org/apache/hadoop/hive/common/JavaUtils.java
@@ -71,6 +71,10 @@ public final class JavaUtils {
     return classLoader;
   }
 
+  public static Class loadClass(String shadePrefix, String className) throws ClassNotFoundException {
+    return loadClass(shadePrefix + "." + className);
+  }
+
   public static Class loadClass(String className) throws ClassNotFoundException {
     return loadClass(className, true);
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/66f6748a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
----------------------------------------------------------------------
diff --git a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
index e56c14f..b81c47d 100644
--- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
+++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
@@ -1911,6 +1911,9 @@ public class HiveConf extends Configuration {
       "Hive streaming ingest has auto flush mechanism to flush all open record updaters under memory pressure.\n" +
         "When memory usage exceed hive.heap.memory.monitor.default.usage.threshold, the auto-flush mechanism will \n" +
         "wait until this size (default 100Mb) of records are ingested before triggering flush."),
+    HIVE_CLASSLOADER_SHADE_PREFIX("hive.classloader.shade.prefix", "", "During reflective instantiation of a class\n" +
+      "(input, output formats, serde etc.), when classloader throws ClassNotFoundException, as a fallback this\n" +
+      "shade prefix will be used before class reference and retried."),
 
     HIVE_ORC_MS_FOOTER_CACHE_ENABLED("hive.orc.splits.ms.footer.cache.enabled", false,
         "Whether to enable using file metadata cache in metastore for ORC file footers."),

http://git-wip-us.apache.org/repos/asf/hive/blob/66f6748a/streaming/src/java/org/apache/hive/streaming/AbstractRecordWriter.java
----------------------------------------------------------------------
diff --git a/streaming/src/java/org/apache/hive/streaming/AbstractRecordWriter.java b/streaming/src/java/org/apache/hive/streaming/AbstractRecordWriter.java
index 685e0cc..b6c8890 100644
--- a/streaming/src/java/org/apache/hive/streaming/AbstractRecordWriter.java
+++ b/streaming/src/java/org/apache/hive/streaming/AbstractRecordWriter.java
@@ -128,31 +128,41 @@ public abstract class AbstractRecordWriter implements RecordWriter {
     if (conn == null) {
       throw new StreamingException("Streaming connection cannot be null during record writer initialization");
     }
+    this.conn = conn;
+    this.curBatchMinWriteId = minWriteId;
+    this.curBatchMaxWriteId = maxWriteId;
+    this.conf = conn.getHiveConf();
+    this.defaultPartitionName = conf.getVar(HiveConf.ConfVars.DEFAULTPARTITIONNAME);
+    this.table = conn.getTable();
+    this.inputColumns = table.getSd().getCols().stream().map(FieldSchema::getName).collect(Collectors.toList());
+    this.inputTypes = table.getSd().getCols().stream().map(FieldSchema::getType).collect(Collectors.toList());
+    if (conn.isPartitionedTable() && conn.isDynamicPartitioning()) {
+      this.partitionColumns = table.getPartitionKeys().stream().map(FieldSchema::getName)
+        .collect(Collectors.toList());
+      this.inputColumns.addAll(partitionColumns);
+      this.inputTypes
+        .addAll(table.getPartitionKeys().stream().map(FieldSchema::getType).collect(Collectors.toList()));
+    }
+    this.fullyQualifiedTableName = Warehouse.getQualifiedName(table.getDbName(), table.getTableName());
+    String outFormatName = this.table.getSd().getOutputFormat();
     try {
-      this.conn = conn;
-      this.curBatchMinWriteId = minWriteId;
-      this.curBatchMaxWriteId = maxWriteId;
-      this.conf = conn.getHiveConf();
-      this.defaultPartitionName = conf.getVar(HiveConf.ConfVars.DEFAULTPARTITIONNAME);
-      this.table = conn.getTable();
-      this.inputColumns = table.getSd().getCols().stream().map(FieldSchema::getName).collect(Collectors.toList());
-      this.inputTypes = table.getSd().getCols().stream().map(FieldSchema::getType).collect(Collectors.toList());
-      if (conn.isPartitionedTable() && conn.isDynamicPartitioning()) {
-        this.partitionColumns = table.getPartitionKeys().stream().map(FieldSchema::getName)
-          .collect(Collectors.toList());
-        this.inputColumns.addAll(partitionColumns);
-        this.inputTypes
-          .addAll(table.getPartitionKeys().stream().map(FieldSchema::getType).collect(Collectors.toList()));
-      }
-      this.fullyQualifiedTableName = Warehouse.getQualifiedName(table.getDbName(), table.getTableName());
-      String outFormatName = this.table.getSd().getOutputFormat();
       this.acidOutputFormat = (AcidOutputFormat<?, ?>) ReflectionUtils
         .newInstance(JavaUtils.loadClass(outFormatName), conf);
-      setupMemoryMonitoring();
     } catch (ClassNotFoundException e) {
-      throw new StreamingException(e.getMessage(), e);
+      String shadePrefix = conf.getVar(HiveConf.ConfVars.HIVE_CLASSLOADER_SHADE_PREFIX);
+      if (shadePrefix != null && !shadePrefix.trim().isEmpty()) {
+        try {
+          LOG.info("Shade prefix: {} specified. Using as fallback to load {}..", shadePrefix, outFormatName);
+          this.acidOutputFormat = (AcidOutputFormat<?, ?>) ReflectionUtils
+            .newInstance(JavaUtils.loadClass(shadePrefix, outFormatName), conf);
+        } catch (ClassNotFoundException e1) {
+          throw new StreamingException(e.getMessage(), e);
+        }
+      } else {
+        throw new StreamingException(e.getMessage(), e);
+      }
     }
-
+    setupMemoryMonitoring();
     try {
       final AbstractSerDe serDe = createSerde();
       this.inputRowObjectInspector = (StructObjectInspector) serDe.getObjectInspector();


[50/50] [abbrv] hive git commit: HIVE-19561: Update README.md to update requirements for Hadoop and RELEASE_NOTES

Posted by vg...@apache.org.
HIVE-19561: Update README.md to update requirements for Hadoop and RELEASE_NOTES


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

Branch: refs/heads/branch-3.0.0
Commit: f1f26504642ecf7fc1cd8fd046f24920a2f0051b
Parents: 04ad5d1
Author: Vineet Garg <vg...@apache.org>
Authored: Tue May 15 15:15:47 2018 -0700
Committer: Vineet Garg <vg...@apache.org>
Committed: Tue May 15 15:32:34 2018 -0700

----------------------------------------------------------------------
 README.md         |    2 +-
 RELEASE_NOTES.txt | 1745 ++++++++++++++++++++++++++++++++++++++++++++++++
 2 files changed, 1746 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/f1f26504/README.md
----------------------------------------------------------------------
diff --git a/README.md b/README.md
index f7a4f46..849b72d 100644
--- a/README.md
+++ b/README.md
@@ -77,7 +77,7 @@ Requirements
 
 - Java 1.7 or 1.8
 
-- Hadoop 1.x, 2.x (2.x required for Hive 2.x)
+- Hadoop 1.x, 2.x, 3.x (3.x required for Hive 3.x)
 
 
 Upgrading from older versions of Hive


[46/50] [abbrv] hive git commit: Revert "HIVE-19250: Schema column definition inconsistencies in MySQL (Naveen Gangam, reviewed by Aihua Xu)"

Posted by vg...@apache.org.
Revert "HIVE-19250: Schema column definition inconsistencies in MySQL (Naveen Gangam, reviewed by Aihua Xu)"

This reverts commit efe9ab8ed6456f7a3e53c643974d4b7822ea3707.


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

Branch: refs/heads/branch-3.0.0
Commit: dc83c6f4b1121d73e3e54a01105e99bcff5189d5
Parents: efe9ab8
Author: Vineet Garg <vg...@apache.org>
Authored: Tue May 15 14:43:42 2018 -0700
Committer: Vineet Garg <vg...@apache.org>
Committed: Tue May 15 14:43:42 2018 -0700

----------------------------------------------------------------------
 metastore/scripts/upgrade/mysql/039-HIVE-12274.mysql.sql | 10 +++++-----
 1 file changed, 5 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/dc83c6f4/metastore/scripts/upgrade/mysql/039-HIVE-12274.mysql.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/mysql/039-HIVE-12274.mysql.sql b/metastore/scripts/upgrade/mysql/039-HIVE-12274.mysql.sql
index bd6bd29..cdaf286 100644
--- a/metastore/scripts/upgrade/mysql/039-HIVE-12274.mysql.sql
+++ b/metastore/scripts/upgrade/mysql/039-HIVE-12274.mysql.sql
@@ -1,14 +1,14 @@
 ALTER TABLE COLUMNS_V2 MODIFY TYPE_NAME MEDIUMTEXT;
-ALTER TABLE TABLE_PARAMS MODIFY PARAM_VALUE MEDIUMTEXT CHARACTER SET latin1 COLLATE latin1_bin;
-ALTER TABLE SERDE_PARAMS MODIFY PARAM_VALUE MEDIUMTEXT CHARACTER SET latin1 COLLATE latin1_bin;
-ALTER TABLE SD_PARAMS MODIFY PARAM_VALUE MEDIUMTEXT CHARACTER SET latin1 COLLATE latin1_bin;
+ALTER TABLE TABLE_PARAMS MODIFY PARAM_VALUE MEDIUMTEXT;
+ALTER TABLE SERDE_PARAMS MODIFY PARAM_VALUE MEDIUMTEXT;
+ALTER TABLE SD_PARAMS MODIFY PARAM_VALUE MEDIUMTEXT;
 
 ALTER TABLE TBLS MODIFY TBL_NAME varchar(256) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL;
-ALTER TABLE NOTIFICATION_LOG MODIFY TBL_NAME varchar(256);
+ALTER TABLE NOTIFICATION_LOG MODIFY TBL_NAME varchar(256) CHARACTER SET latin1 COLLATE latin1_bin;
 ALTER TABLE PARTITION_EVENTS MODIFY TBL_NAME varchar(256) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL;
 ALTER TABLE TAB_COL_STATS MODIFY TABLE_NAME varchar(256) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL;
 ALTER TABLE PART_COL_STATS MODIFY TABLE_NAME varchar(256) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL;
-ALTER TABLE COMPLETED_TXN_COMPONENTS MODIFY CTC_TABLE varchar(256);
+ALTER TABLE COMPLETED_TXN_COMPONENTS MODIFY CTC_TABLE varchar(256) CHARACTER SET latin1 COLLATE latin1_bin;
 
 ALTER TABLE COLUMNS_V2 MODIFY COLUMN_NAME varchar(767) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL;
 ALTER TABLE PART_COL_PRIVS MODIFY COLUMN_NAME varchar(767) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL;


[25/50] [abbrv] hive git commit: HIVE-19506: Test suites timing out (Vineet Garg, reviewed by Vihang Karajgaonkar)

Posted by vg...@apache.org.
HIVE-19506: Test suites timing out (Vineet Garg, reviewed by Vihang Karajgaonkar)


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

Branch: refs/heads/branch-3.0.0
Commit: 2e5f8635a3b6d510c29b6eb0f418ae171e9d6ef5
Parents: 1b917c2
Author: Vineet Garg <vg...@apache.org>
Authored: Sun May 13 12:12:45 2018 -0700
Committer: Vineet Garg <vg...@apache.org>
Committed: Sun May 13 14:35:59 2018 -0700

----------------------------------------------------------------------
 .../apache/hive/hcatalog/mapreduce/TestSequenceFileReadWrite.java | 3 +++
 1 file changed, 3 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/2e5f8635/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/mapreduce/TestSequenceFileReadWrite.java
----------------------------------------------------------------------
diff --git a/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/mapreduce/TestSequenceFileReadWrite.java b/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/mapreduce/TestSequenceFileReadWrite.java
index ffd7146..4ebec1b 100644
--- a/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/mapreduce/TestSequenceFileReadWrite.java
+++ b/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/mapreduce/TestSequenceFileReadWrite.java
@@ -25,6 +25,7 @@ import static org.junit.Assert.assertTrue;
 import java.io.File;
 import java.io.IOException;
 import java.util.ArrayList;
+import org.junit.Ignore;
 import java.util.Iterator;
 
 import org.apache.commons.io.FileUtils;
@@ -160,6 +161,7 @@ public class TestSequenceFileReadWrite {
     assertEquals(input.length, numTuplesRead);
   }
 
+  @Ignore("Disabling this test. Check HIVE-19506 for more details")
   @Test
   public void testSequenceTableWriteReadMR() throws Exception {
     String createTable = "CREATE TABLE demo_table_2(a0 int, a1 String, a2 String) STORED AS SEQUENCEFILE";
@@ -206,6 +208,7 @@ public class TestSequenceFileReadWrite {
     assertEquals(input.length, numTuplesRead);
   }
 
+  @Ignore("Disabling this test. Check HIVE-19506 for more details")
   @Test
   public void testTextTableWriteReadMR() throws Exception {
     String createTable = "CREATE TABLE demo_table_3(a0 int, a1 String, a2 String) STORED AS TEXTFILE";


[19/50] [abbrv] hive git commit: HIVE-19483: Metastore cleaner tasks that run periodically are created more than once (Jesus Camacho Rodriguez, reviewed by Alan Gates)

Posted by vg...@apache.org.
HIVE-19483: Metastore cleaner tasks that run periodically are created more than once (Jesus Camacho Rodriguez, reviewed by Alan Gates)


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

Branch: refs/heads/branch-3.0.0
Commit: 1a31e4919dc5fd2c23e6d2460580fb21bcb2de20
Parents: e40aa04
Author: Jesus Camacho Rodriguez <jc...@apache.org>
Authored: Fri May 11 08:53:53 2018 -0700
Committer: Jesus Camacho Rodriguez <jc...@apache.org>
Committed: Fri May 11 09:21:20 2018 -0700

----------------------------------------------------------------------
 .../hadoop/hive/metastore/HiveMetaStore.java    | 33 ++++++++++++--------
 1 file changed, 20 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/1a31e491/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
index 966b7a9..f9bd64b 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
@@ -242,6 +242,11 @@ public class HiveMetaStore extends ThriftHiveMetastore {
     private final Configuration conf; // stores datastore (jpox) properties,
                                      // right now they come from jpox.properties
 
+    // Flag to control that always threads are initialized only once
+    // instead of multiple times
+    private final static AtomicBoolean alwaysThreadsInitialized =
+        new AtomicBoolean(false);
+
     private static String currentUrl;
     private FileMetadataManager fileMetadataManager;
     private PartitionExpressionProxy expressionProxy;
@@ -559,19 +564,21 @@ public class HiveMetaStore extends ThriftHiveMetastore {
         partitionValidationPattern = null;
       }
 
-      ThreadPool.initialize(conf);
-      Collection<String> taskNames =
-          MetastoreConf.getStringCollection(conf, ConfVars.TASK_THREADS_ALWAYS);
-      for (String taskName : taskNames) {
-        MetastoreTaskThread task =
-            JavaUtils.newInstance(JavaUtils.getClass(taskName, MetastoreTaskThread.class));
-        task.setConf(conf);
-        long freq = task.runFrequency(TimeUnit.MILLISECONDS);
-        // For backwards compatibility, since some threads used to be hard coded but only run if
-        // frequency was > 0
-        if (freq > 0) {
-          ThreadPool.getPool().scheduleAtFixedRate(task, freq, freq, TimeUnit.MILLISECONDS);
-
+      // We only initialize once the tasks that need to be run periodically
+      if (alwaysThreadsInitialized.compareAndSet(false, true)) {
+        ThreadPool.initialize(conf);
+        Collection<String> taskNames =
+            MetastoreConf.getStringCollection(conf, ConfVars.TASK_THREADS_ALWAYS);
+        for (String taskName : taskNames) {
+          MetastoreTaskThread task =
+              JavaUtils.newInstance(JavaUtils.getClass(taskName, MetastoreTaskThread.class));
+          task.setConf(conf);
+          long freq = task.runFrequency(TimeUnit.MILLISECONDS);
+          // For backwards compatibility, since some threads used to be hard coded but only run if
+          // frequency was > 0
+          if (freq > 0) {
+            ThreadPool.getPool().scheduleAtFixedRate(task, freq, freq, TimeUnit.MILLISECONDS);
+          }
         }
       }
       expressionProxy = PartFilterExprUtil.createExpressionProxy(conf);


[38/50] [abbrv] hive git commit: HIVE-19410 : don't create serde reader in LLAP if there's no cache (Sergey Shelukhin, reviewed by Prasanth Jayachandran)

Posted by vg...@apache.org.
HIVE-19410 : don't create serde reader in LLAP if there's no cache (Sergey Shelukhin, reviewed by Prasanth Jayachandran)


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

Branch: refs/heads/branch-3.0.0
Commit: d33f79fe5307fa7044efaba64d666acf9be3a72a
Parents: 14d0690
Author: sergey <se...@apache.org>
Authored: Mon May 7 12:09:04 2018 -0700
Committer: Matt McCline <mm...@hortonworks.com>
Committed: Mon May 14 14:11:34 2018 -0500

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hive/llap/io/api/impl/LlapIoImpl.java  | 3 ++-
 .../hadoop/hive/llap/io/encoded/SerDeEncodedDataReader.java       | 1 +
 2 files changed, 3 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/d33f79fe/llap-server/src/java/org/apache/hadoop/hive/llap/io/api/impl/LlapIoImpl.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/io/api/impl/LlapIoImpl.java b/llap-server/src/java/org/apache/hadoop/hive/llap/io/api/impl/LlapIoImpl.java
index 747b399..5003d9b 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/io/api/impl/LlapIoImpl.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/io/api/impl/LlapIoImpl.java
@@ -133,7 +133,8 @@ public class LlapIoImpl implements LlapIo<VectorizedRowBatch> {
     MetadataCache metadataCache = null;
     SerDeLowLevelCacheImpl serdeCache = null; // TODO: extract interface when needed
     BufferUsageManager bufferManagerOrc = null, bufferManagerGeneric = null;
-    boolean isEncodeEnabled = HiveConf.getBoolVar(conf, ConfVars.LLAP_IO_ENCODE_ENABLED);
+    boolean isEncodeEnabled = useLowLevelCache
+        && HiveConf.getBoolVar(conf, ConfVars.LLAP_IO_ENCODE_ENABLED);
     if (useLowLevelCache) {
       // Memory manager uses cache policy to trigger evictions, so create the policy first.
       boolean useLrfu = HiveConf.getBoolVar(conf, HiveConf.ConfVars.LLAP_USE_LRFU);

http://git-wip-us.apache.org/repos/asf/hive/blob/d33f79fe/llap-server/src/java/org/apache/hadoop/hive/llap/io/encoded/SerDeEncodedDataReader.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/io/encoded/SerDeEncodedDataReader.java b/llap-server/src/java/org/apache/hadoop/hive/llap/io/encoded/SerDeEncodedDataReader.java
index 8b89ae9..1cfe929 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/io/encoded/SerDeEncodedDataReader.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/io/encoded/SerDeEncodedDataReader.java
@@ -178,6 +178,7 @@ public class SerDeEncodedDataReader extends CallableWithNdc<Void>
       InputFormat<?, ?> sourceInputFormat, Deserializer sourceSerDe,
       QueryFragmentCounters counters, TypeDescription schema, Map<Path, PartitionDesc> parts)
           throws IOException {
+    assert cache != null;
     this.cache = cache;
     this.bufferManager = bufferManager;
     this.bufferFactory = new BufferObjectFactory() {


[49/50] [abbrv] hive git commit: HIVE-19561: Update README.md to update requirements for Hadoop and RELEASE_NOTES

Posted by vg...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/f1f26504/RELEASE_NOTES.txt
----------------------------------------------------------------------
diff --git a/RELEASE_NOTES.txt b/RELEASE_NOTES.txt
index d8e527b..f2f2b8c 100644
--- a/RELEASE_NOTES.txt
+++ b/RELEASE_NOTES.txt
@@ -1,3 +1,1748 @@
+Release Notes - Hive - Version 3.0.0
+
+** Sub-task
+    * [HIVE-11133] - Support hive.explain.user for Spark
+    * [HIVE-11418] - Dropping a database in an encryption zone with CASCADE and trash enabled fails
+    * [HIVE-13567] - Enable auto-gather column stats by default
+    * [HIVE-13583] - E061-14: Search Conditions
+    * [HIVE-13673] - LLAP: handle case where no service instance is found on the host specified in the input split
+    * [HIVE-14412] - Add timestamp with time zone
+    * [HIVE-14487] - Add REBUILD statement for materialized views
+    * [HIVE-14495] - Add SHOW MATERIALIZED VIEWS statement
+    * [HIVE-14498] - Freshness period for query rewriting using materialized views
+    * [HIVE-14518] - Support 'having' translation for Druid GroupBy queries
+    * [HIVE-14747] - Remove JAVA paths from profiles by sending them from ptest-client
+    * [HIVE-14947] - Add support for Acid 2 in Merge
+    * [HIVE-15016] - Run tests with Hadoop 3.0.0-beta1
+    * [HIVE-15018] - ALTER rewriting flag in materialized view 
+    * [HIVE-15051] - Test framework integration with findbugs, rat checks etc.
+    * [HIVE-15173] - Allow dec as an alias for decimal
+    * [HIVE-15212] - merge branch into master
+    * [HIVE-15326] - Hive shims report Unrecognized Hadoop major version number: 3.0.0-alpha2-SNAPSHOT
+    * [HIVE-15436] - Enhancing metastore APIs to retrieve only materialized views
+    * [HIVE-15490] - REPL LOAD & DUMP support for INSERT events with change management
+    * [HIVE-15619] - Column pruner should handle DruidQuery
+    * [HIVE-15642] - Replicate Insert Overwrites, Dynamic Partition Inserts and Loads
+    * [HIVE-15673] - Allow multiple queries with disjunction
+    * [HIVE-15705] - Event replication for constraints
+    * [HIVE-15725] - Make it possible to run checkstyle for a specific module
+    * [HIVE-15758] - Allow correlated scalar subqueries with aggregates which has non-equi join predicates
+    * [HIVE-15834] - Add unit tests for org.json usage on master
+    * [HIVE-15899] - Make CTAS with acid target table and insert into acid_tbl select ... union all ... work
+    * [HIVE-15939] - Make cast expressions comply more to sql2011
+    * [HIVE-15982] - Support the width_bucket function
+    * [HIVE-15986] - Support "is [not] distinct from"
+    * [HIVE-16171] - Support replication of truncate table
+    * [HIVE-16186] - REPL DUMP shows last event ID of the database even if we use LIMIT option.
+    * [HIVE-16197] - Incremental insert into a partitioned table doesn't get replicated.
+    * [HIVE-16207] - Add support for Complex Types in Fast SerDe
+    * [HIVE-16228] - Support subqueries in complex expression in SELECT clause
+    * [HIVE-16256] - Flaky test: TestCliDriver.testCliDriver[comments]
+    * [HIVE-16266] - Enable function metadata to be written during bootstrap
+    * [HIVE-16267] - Enable bootstrap function metadata to be loaded in repl load
+    * [HIVE-16268] - enable incremental repl dump to handle functions metadata
+    * [HIVE-16269] - enable incremental function dump to be loaded via repl load 
+    * [HIVE-16272] - support for drop function in incremental replication 
+    * [HIVE-16276] - Fix NoSuchMethodError: com.amazonaws.services.s3.transfer.TransferManagerConfiguration.setMultipartUploadThreshold(I)V
+    * [HIVE-16294] - Support snapshot for truncate table
+    * [HIVE-16312] - Flaky test: TestHCatClient.testTransportFailure
+    * [HIVE-16313] - Flaky test: TestBeeLineDriver[drop_with_concurrency]
+    * [HIVE-16320] - Flaky test: TestBeeLineDriver.testCliDriver[escape_comments]
+    * [HIVE-16330] - Improve plans for scalar subquery with aggregates
+    * [HIVE-16344] - Test and support replication of exchange partition
+    * [HIVE-16372] - Enable DDL statement for non-native tables (add/remove table properties)
+    * [HIVE-16400] - Fix the MDC reference to use slf4j rather than log4j
+    * [HIVE-16416] - Service: move constants out from HiveAuthFactory
+    * [HIVE-16467] - Flaky test: TestCliDriver.testCliDriver[vector_order_null]
+    * [HIVE-16488] - Support replicating into existing db if the db is empty
+    * [HIVE-16493] - Skip column stats when colStats is empty
+    * [HIVE-16504] - Addition of binary licenses broke rat check
+    * [HIVE-16530] - Add HS2 operation logs and improve logs for REPL commands
+    * [HIVE-16532] - HIVE on hadoop 3 build failed due to hdfs client/server jar separation
+    * [HIVE-16535] - Hive fails to build from source code tarball
+    * [HIVE-16542] - make merge that targets acid 2.0 table fail-fast 
+    * [HIVE-16555] - Add a new thrift API call for get_metastore_uuid
+    * [HIVE-16556] - Modify schematool scripts to initialize and create METASTORE_DB_PROPERTIES table
+    * [HIVE-16566] - Set column stats default as true when creating new tables/partitions
+    * [HIVE-16568] - Support complex types in external LLAP InputFormat
+    * [HIVE-16579] - CachedStore: improvements to partition col stats caching and cache column stats for unpartitioned table
+    * [HIVE-16586] - Fix Unit test failures when CachedStore is enabled
+    * [HIVE-16591] - DR for function Binaries on HDFS 
+    * [HIVE-16600] - Refactor SetSparkReducerParallelism#needSetParallelism to enable parallel order by in multi_insert cases
+    * [HIVE-16601] - Display Session Id and Query Name / Id in Spark UI
+    * [HIVE-16617] - Clean up javadoc from errors in module hive-shims
+    * [HIVE-16618] - Clean up javadoc from errors in module hive-common
+    * [HIVE-16619] - Clean up javadoc from errors in module hive-serde
+    * [HIVE-16628] - Fix query25 when it uses a mix of MergeJoin and MapJoin
+    * [HIVE-16637] - Improve end-of-data checking for LLAP input format
+    * [HIVE-16642] - New Events created as part of replv2 potentially break replv1
+    * [HIVE-16644] - Hook Change Manager to Insert Overwrite
+    * [HIVE-16647] - Improve the validation output to make the output to stderr and stdout more consistent
+    * [HIVE-16651] - LlapProtocolClientProxy stack trace when using llap input format
+    * [HIVE-16652] - LlapInputFormat: Seeing "output error" WARN message
+    * [HIVE-16653] - Mergejoin should give itself a correct tag
+    * [HIVE-16672] - Parquet vectorization doesn't work for tables with partition info
+    * [HIVE-16684] - Bootstrap REPL DUMP shouldn't fail when table is dropped after fetching the table names.
+    * [HIVE-16686] - repl invocations of distcp needs additional handling
+    * [HIVE-16688] - Make sure Alter Table to set transaction=true acquires X lock
+    * [HIVE-16691] - Add test for more datatypes for LlapInputFormat
+    * [HIVE-16697] - Schema table validator should return a sorted list of missing tables 
+    * [HIVE-16702] - Use LazyBinarySerDe for LLAP InputFormat
+    * [HIVE-16706] - Bootstrap REPL DUMP shouldn't fail when a partition is dropped/renamed when dump in progress.
+    * [HIVE-16714] - make Task Dependency on Repl Load more intuitive
+    * [HIVE-16715] - Clean up javadoc from errors in modules llap-client, metastore, spark-client
+    * [HIVE-16722] - Converting bucketed non-acid table to acid should perform validation
+    * [HIVE-16727] - REPL DUMP for insert event should't fail if the table is already dropped.
+    * [HIVE-16729] - Improve location validator to check for blank paths.
+    * [HIVE-16747] - Remove YETUS*.sh files after a YETUS release
+    * [HIVE-16748] - Integreate YETUS to Pre-Commit
+    * [HIVE-16750] - Support change management for rename table/partition.
+    * [HIVE-16764] - Support numeric as same as decimal
+    * [HIVE-16765] - ParquetFileReader should be closed to avoid resource leak
+    * [HIVE-16774] - Support position in ORDER BY when using SELECT *
+    * [HIVE-16775] - Fix HiveFilterAggregateTransposeRule when filter is always false
+    * [HIVE-16779] - CachedStore leak PersistenceManager resources
+    * [HIVE-16782] - Flaky Test: TestMiniLlapLocalCliDriver[subquery_scalar]
+    * [HIVE-16785] - Ensure replication actions are idempotent if any series of events are applied again.
+    * [HIVE-16797] - Enhance HiveFilterSetOpTransposeRule to remove union branches
+    * [HIVE-16813] - Incremental REPL LOAD should load the events in the same sequence as it is dumped.
+    * [HIVE-16827] - Merge stats task and column stats task into a single task
+    * [HIVE-16837] - MetadataOnly optimizer conflicts with count distinct rewrite
+    * [HIVE-16838] - Improve plans for subqueries with non-equi co-related predicates
+    * [HIVE-16848] - NPE during CachedStore refresh
+    * [HIVE-16892] - Move creation of _files from ReplCopyTask to analysis phase for boostrap replication 
+    * [HIVE-16893] - move replication dump related work in semantic analysis phase to execution phase using a task
+    * [HIVE-16895] -  Multi-threaded execution of bootstrap dump of partitions
+    * [HIVE-16896] - move replication load related work in semantic analysis phase to execution phase using a task
+    * [HIVE-16901] - Distcp optimization - One distcp per ReplCopyTask 
+    * [HIVE-16912] - Improve table validator's performance against Oracle
+    * [HIVE-16926] - LlapTaskUmbilicalExternalClient should not start new umbilical server for every fragment request
+    * [HIVE-16974] - Change the sort key for the schema tool validator to be <ID>
+    * [HIVE-16981] - hive.optimize.bucketingsorting should compare the schema before removing RS
+    * [HIVE-16990] - REPL LOAD should update last repl ID only after successful copy of data files.
+    * [HIVE-16992] - LLAP: monitoring and better default lambda for LRFU policy
+    * [HIVE-16996] - Add HLL as an alternative to FM sketch to compute stats
+    * [HIVE-16997] - Extend object store to store and use bit vectors
+    * [HIVE-16998] - Add config to enable HoS DPP only for map-joins
+    * [HIVE-17005] - Ensure REPL DUMP and REPL LOAD are authorized properly
+    * [HIVE-17021] - Support replication of concatenate operation.
+    * [HIVE-17087] - Remove unnecessary HoS DPP trees during map-join conversion
+    * [HIVE-17091] - "Timed out getting readerEvents" error from external LLAP client
+    * [HIVE-17100] - Improve HS2 operation logs for REPL commands.
+    * [HIVE-17112] - Reduce logging in HiveSparkClientFactory and RemoteHiveSparkClient
+    * [HIVE-17132] - Add InterfaceAudience and InterfaceStability annotations for UDF APIs
+    * [HIVE-17137] - Fix javolution conflict
+    * [HIVE-17153] - Flaky test: TestMiniSparkOnYarnCliDriver[spark_dynamic_partition_pruning]
+    * [HIVE-17157] - Add InterfaceAudience and InterfaceStability annotations for ObjectInspector APIs
+    * [HIVE-17167] - Create metastore specific configuration tool
+    * [HIVE-17168] - Create separate module for stand alone metastore
+    * [HIVE-17170] - Move thrift generated code to stand alone metastore
+    * [HIVE-17178] - Spark Partition Pruning Sink Operator can't target multiple Works
+    * [HIVE-17183] - Disable rename operations during bootstrap dump
+    * [HIVE-17185] - TestHiveMetaStoreStatsMerge.testStatsMerge is failing
+    * [HIVE-17195] - Long chain of tasks created by REPL LOAD shouldn't cause stack corruption.
+    * [HIVE-17196] - CM: ReplCopyTask should retain the original file names even if copied from CM path.
+    * [HIVE-17205] - add functional support for unbucketed tables
+    * [HIVE-17212] - Dynamic add partition by insert shouldn't generate INSERT event.
+    * [HIVE-17214] - check/fix conversion of unbucketed non-acid to acid
+    * [HIVE-17215] - Streaming Ingest API writing unbucketed tables
+    * [HIVE-17216] - Additional qtests for HoS DPP
+    * [HIVE-17224] - Move JDO classes to standalone metastore
+    * [HIVE-17225] - HoS DPP pruning sink ops can target parallel work objects
+    * [HIVE-17241] - Change metastore classes to not use the shims
+    * [HIVE-17247] - HoS DPP: UDFs on the partition column side does not evaluate correctly
+    * [HIVE-17256] - add a notion of a guaranteed task to LLAP
+    * [HIVE-17289] - EXPORT and IMPORT shouldn't perform distcp with doAs privileged user.
+    * [HIVE-17292] - Change TestMiniSparkOnYarnCliDriver test configuration to use the configured cores
+    * [HIVE-17297] - allow AM to use LLAP guaranteed tasks
+    * [HIVE-17307] - Change the metastore to not use the metrics code in hive/common
+    * [HIVE-17316] - Use String.startsWith for the hidden configuration variables
+    * [HIVE-17318] - Make Hikari CP configurable using hive properties in hive-site.xml
+    * [HIVE-17319] - Make BoneCp configurable using hive properties in hive-site.xml
+    * [HIVE-17330] - refactor TezSessionPoolManager to separate its multiple functions
+    * [HIVE-17346] - TestMiniSparkOnYarnCliDriver[spark_dynamic_partition_pruning] is failing every time
+    * [HIVE-17347] - TestMiniSparkOnYarnCliDriver[spark_dynamic_partition_pruning_mapjoin_only] is failing every time
+    * [HIVE-17359] - Deal with TypeInfo dependencies in the metastore
+    * [HIVE-17371] - Move tokenstores to metastore module
+    * [HIVE-17375] - stddev_samp,var_samp standard compliance
+    * [HIVE-17380] - refactor LlapProtocolClientProxy to be usable with other protocols
+    * [HIVE-17381] - When we enable Parquet Writer Version V2, hive throws an exception: Unsupported encoding: DELTA_BYTE_ARRAY.
+    * [HIVE-17382] - Change startsWith relation introduced in HIVE-17316
+    * [HIVE-17387] - implement Tez AM registry in Hive
+    * [HIVE-17405] - HoS DPP ConstantPropagate should use ConstantPropagateOption.SHORTCUT
+    * [HIVE-17409] - refactor LLAP ZK registry to make the ZK-registry part reusable
+    * [HIVE-17414] - HoS DPP + Vectorization generates invalid explain plan due to CombineEquivalentWorkResolver
+    * [HIVE-17428] - REPL LOAD of ALTER_PARTITION event doesn't create import tasks if the partition doesn't exist during analyze phase.
+    * [HIVE-17455] - External LLAP client: connection to HS2 should be kept open until explicitly closed
+    * [HIVE-17456] - Set current database for external LLAP interface
+    * [HIVE-17473] - implement workload management pools
+    * [HIVE-17482] - External LLAP client: acquire locks for tables queried directly by LLAP
+    * [HIVE-17488] - Move first set of classes to standalone metastore
+    * [HIVE-17494] - Bootstrap REPL DUMP throws exception if a partitioned table is dropped while reading partitions.
+    * [HIVE-17495] - CachedStore: prewarm improvement (avoid multiple sql calls to read partition column stats), refactoring and caching some aggregate stats
+    * [HIVE-17506] - Fix standalone-metastore pom.xml to not depend on hive's main pom
+    * [HIVE-17508] - Implement global execution triggers based on counters
+    * [HIVE-17514] - Use SHA-256 for cookie signer to improve security
+    * [HIVE-17515] - Use SHA-256 for GenericUDFMaskHash to improve security
+    * [HIVE-17527] - Support replication for rename/move table across database
+    * [HIVE-17528] - Add more q-tests for Hive-on-Spark with Parquet vectorized reader
+    * [HIVE-17534] - Add a config to turn off parquet vectorization
+    * [HIVE-17537] - Move Warehouse class to standalone metastore
+    * [HIVE-17541] - Move testing related methods from MetaStoreUtils to some testing related utility
+    * [HIVE-17566] - Create schema required for workload management.
+    * [HIVE-17581] - Replace some calcite dependencies with native ones
+    * [HIVE-17607] - remove ColumnStatsDesc usage from columnstatsupdatetask
+    * [HIVE-17608] - REPL LOAD should overwrite the data files if exists instead of duplicating it
+    * [HIVE-17617] - Rollup of an empty resultset should contain the grouping of the empty grouping set
+    * [HIVE-17629] - CachedStore - wait for prewarm at use time, not init time
+    * [HIVE-17645] - MM tables patch conflicts with HIVE-17482 (Spark/Acid integration)
+    * [HIVE-17647] - DDLTask.generateAddMmTasks(Table tbl) and other random code should not start transactions
+    * [HIVE-17651] - TableScanOperator might miss vectorization on flag
+    * [HIVE-17652] - retire ANALYZE TABLE ... PARTIALSCAN
+    * [HIVE-17661] - DBTxnManager.acquireLocks() - MM tables should use shared lock for Insert
+    * [HIVE-17671] - TableScanDesc.isAcidTable is restricted to FullAcid tables
+    * [HIVE-17681] - Need to log bootstrap dump progress state property to HS2 logs.
+    * [HIVE-17692] - Block HCat on Acid tables
+    * [HIVE-17696] - Vectorized reader does not seem to be pushing down projection columns in certain code paths
+    * [HIVE-17698] - FileSinkDesk.getMergeInputDirName() uses stmtId=0
+    * [HIVE-17708] - Upgrade surefire to 2.20.1
+    * [HIVE-17728] - TestHCatClient should use hive.metastore.transactional.event.listeners as per recommendation.
+    * [HIVE-17733] - Move RawStore to standalone metastore
+    * [HIVE-17743] - Add InterfaceAudience and InterfaceStability annotations for Thrift generated APIs
+    * [HIVE-17748] - ReplCopyTask doesn't support multi-file CopyWork
+    * [HIVE-17750] - add a flag to automatically create most tables as MM 
+    * [HIVE-17756] - Enable subquery related Qtests for Hive on Spark
+    * [HIVE-17757] - REPL LOAD need to use customised configurations to execute distcp/remote copy.
+    * [HIVE-17771] - Implement commands to manage resource plan
+    * [HIVE-17778] - Add support for custom counters in trigger expression
+    * [HIVE-17809] - Implement per pool trigger validation and move sessions across pools
+    * [HIVE-17812] - Move remaining classes that HiveMetaStore depends on 
+    * [HIVE-17835] - HS2 Logs print unnecessary stack trace when HoS query is cancelled
+    * [HIVE-17837] - Explicitly check if the HoS Remote Driver has been lost in the RemoteSparkJobMonitor 
+    * [HIVE-17841] - implement applying the resource plan
+    * [HIVE-17842] - Run checkstyle on ptest2 module with proper configuration
+    * [HIVE-17850] - can VectorizedOrcAcidRowReader be removed once HIVE-17458 is done?
+    * [HIVE-17856] - MM tables - IOW is not ACID compliant
+    * [HIVE-17858] - MM - some union cases are broken
+    * [HIVE-17874] - Parquet vectorization fails on tables with complex columns when there are no projected columns
+    * [HIVE-17884] - Implement create, alter and drop workload management triggers
+    * [HIVE-17887] - Incremental REPL LOAD with Drop partition event on timestamp type partition column fails.
+    * [HIVE-17888] - Display the reason for query cancellation
+    * [HIVE-17897] - "repl load" in bootstrap phase fails when partitions have whitespace
+    * [HIVE-17902] - add notions of default pool and start adding unmanaged mapping
+    * [HIVE-17904] - handle internal Tez AM restart in registry and WM
+    * [HIVE-17905] - propagate background LLAP cluster changes to WM
+    * [HIVE-17906] - use kill query mechanics to kill queries in WM
+    * [HIVE-17907] - enable and apply resource plan commands in HS2
+    * [HIVE-17913] - Cleanup unused methods in Driver
+    * [HIVE-17926] - Support triggers for non-pool sessions
+    * [HIVE-17929] - Use sessionId for HoS Remote Driver Client id
+    * [HIVE-17931] - Implement Parquet vectorization reader for Array type
+    * [HIVE-17933] - make antlr output directory to use a top-level sourceset
+    * [HIVE-17934] - Merging Statistics are promoted to COMPLETE (most of the time)
+    * [HIVE-17945] - Support column projection for index access when using Parquet Vectorization
+    * [HIVE-17950] - Implement resource plan fetching from metastore
+    * [HIVE-17954] - Implement pool, user, group and trigger to pool management API's.
+    * [HIVE-17961] - NPE during initialization of VectorizedParquetRecordReader when input split is null
+    * [HIVE-17967] - Move HiveMetaStore class
+    * [HIVE-17970] - MM LOAD DATA with OVERWRITE doesn't use base_n directory concept
+    * [HIVE-17972] - Implement Parquet vectorization reader for Map type
+    * [HIVE-17980] - Move HiveMetaStoreClient plus a few remaining classes.
+    * [HIVE-17981] - Create a set of builders for Thrift classes
+    * [HIVE-17982] - Move metastore specific itests
+    * [HIVE-17983] - Make the standalone metastore generate tarballs etc.
+    * [HIVE-17990] - Add Thrift and DB storage for Schema Registry objects
+    * [HIVE-17991] - Remove CommandNeedRetryException
+    * [HIVE-17995] - Run checkstyle on standalone-metastore module with proper configuration
+    * [HIVE-17996] - Fix ASF headers
+    * [HIVE-17997] - Add rat plugin and configuration to standalone metastore pom
+    * [HIVE-18002] - add group support for pool mappings
+    * [HIVE-18003] - add explicit jdbc connection string args for mappings
+    * [HIVE-18004] - investigate deriving app name from JDBC connection for pool mapping
+    * [HIVE-18005] - Improve size estimation for array() to be not 0
+    * [HIVE-18025] - Push resource plan changes to tez/unmanaged sessions
+    * [HIVE-18028] - fix WM based on cluster smoke test; add logging
+    * [HIVE-18029] - beeline - support proper usernames based on the URL arg
+    * [HIVE-18031] - Support replication for Alter Database operation.
+    * [HIVE-18034] - Improving logging with HoS executors spend lots of time in GC
+    * [HIVE-18036] - Stats: Remove usage of clone() methods
+    * [HIVE-18053] - Support different table types for MVs
+    * [HIVE-18056] - CachedStore: Have a whitelist/blacklist config to allow selective caching of tables/partitions and allow read while prewarming
+    * [HIVE-18057] - remove PostExecute / PreExecute hook support
+    * [HIVE-18063] - Make CommandProcessorResponse an exception instead of a return class
+    * [HIVE-18071] - add HS2 jmx information about pools and current resource plan
+    * [HIVE-18072] - fix various WM bugs based on cluster testing - part 2
+    * [HIVE-18073] - AM may assert when its guaranteed task count is reduced
+    * [HIVE-18075] - verify commands on a cluster
+    * [HIVE-18076] - killquery doesn't actually work for non-trigger WM kills
+    * [HIVE-18078] - WM getSession needs some retry logic
+    * [HIVE-18084] - Upgrade checkstyle version to support lambdas
+    * [HIVE-18085] - Run checkstyle on storage-api module with proper configuration
+    * [HIVE-18088] - Add WM event traces at query level for debugging
+    * [HIVE-18092] - Fix exception on tables handled by HBaseHandler if columnsstats are auto-gathered
+    * [HIVE-18093] - Improve logging when HoS application is killed
+    * [HIVE-18095] - add a unmanaged flag to triggers (applies to container based sessions)
+    * [HIVE-18096] - add a user-friendly show plan command
+    * [HIVE-18125] - Support arbitrary file names in input to Load Data
+    * [HIVE-18133] - Parametrize TestTxnNoBuckets wrt Vectorization
+    * [HIVE-18134] - some alter resource plan fixes
+    * [HIVE-18138] - Fix columnstats problem in case schema evolution
+    * [HIVE-18141] - Fix StatsUtils.combineRange to combine intervals
+    * [HIVE-18149] - Stats: rownum estimation from datasize underestimates in most cases
+    * [HIVE-18153] - refactor reopen and file management in TezTask
+    * [HIVE-18161] - Remove hive.stats.atomic
+    * [HIVE-18163] - Stats: create materialized view should also collect stats
+    * [HIVE-18170] - User mapping not initialized correctly on start
+    * [HIVE-18179] - Implement validate resource plan (part 1)
+    * [HIVE-18187] - Add jamon generated-sources as source folder
+    * [HIVE-18190] - Consider looking at ORC file schema rather than using _metadata_acid file
+    * [HIVE-18192] - Introduce WriteID per table rather than using global transaction ID
+    * [HIVE-18193] - Migrate existing ACID tables to use write id per table rather than global transaction id
+    * [HIVE-18202] - Automatically migrate hbase.table.name to hbase.mapreduce.hfileoutputformat.table.name for hbase-based table
+    * [HIVE-18203] - change the way WM is enabled and allow dropping the last resource plan
+    * [HIVE-18209] - Fix API call in VectorizedListColumnReader to get value from BytesColumnVector
+    * [HIVE-18211] - Support to read multiple level definition for Map type in Parquet file
+    * [HIVE-18212] - Make sure Yetus check always has a full log
+    * [HIVE-18214] - Flaky test: TestSparkClient
+    * [HIVE-18222] - Update checkstyle rules to be less peeky
+    * [HIVE-18224] - Introduce interface above driver
+    * [HIVE-18229] - add the unmanaged mapping command
+    * [HIVE-18230] - create plan like plan, and replace plan commands for easy modification
+    * [HIVE-18235] - Columnstats gather on mm tables: re-enable disabled test
+    * [HIVE-18237] - missing results for insert_only table after DP insert
+    * [HIVE-18238] - Driver execution may not have configuration changing sideeffects 
+    * [HIVE-18245] - clean up acid_vectorization_original.q
+    * [HIVE-18257] - implement scheduling policy configuration instead of hardcoding fair scheduling
+    * [HIVE-18273] - add LLAP-level counters for WM
+    * [HIVE-18274] - add AM level metrics for WM
+    * [HIVE-18275] - add HS2-level WM metrics
+    * [HIVE-18286] - java.lang.ClassCastException: org.apache.hadoop.hive.ql.exec.vector.DoubleColumnVector cannot be cast to org.apache.hadoop.hive.ql.exec.vector.LongColumnVector
+    * [HIVE-18288] - merge/concat not supported on Acid table
+    * [HIVE-18294] - add switch to make acid table the default
+    * [HIVE-18315] - update tests use non-acid tables
+    * [HIVE-18317] - Improve error messages in TransactionalValidationListerner
+    * [HIVE-18323] - Vectorization: add the support of timestamp in VectorizedPrimitiveColumnReader for parquet
+    * [HIVE-18366] - Update HBaseSerDe to use hbase.mapreduce.hfileoutputformat.table.name instead of hbase.table.name as the table name property
+    * [HIVE-18368] - Improve Spark Debug RDD Graph
+    * [HIVE-18372] - Create testing infra to test different HMS instances
+    * [HIVE-18389] - Print out Spark Web UI URL to the console log
+    * [HIVE-18411] - Fix ArrayIndexOutOfBoundsException for VectorizedListColumnReader
+    * [HIVE-18418] - clean up plugin between DAGs
+    * [HIVE-18437] - use plan parallelism for the default pool if both are present
+    * [HIVE-18438] - WM RP: it's impossible to unset things
+    * [HIVE-18443] - Ensure git gc finished in ptest prep phase before copying repo
+    * [HIVE-18457] - improve show plan output (triggers, mappings)
+    * [HIVE-18458] - Workload manager initializes even when interactive queue is not set
+    * [HIVE-18468] - Create tests to cover alterPartition and renamePartition methods
+    * [HIVE-18478] - Data files deleted from temp table should not be recycled to CM path
+    * [HIVE-18479] - Create tests to cover dropPartition methods
+    * [HIVE-18480] - Create tests for function related methods
+    * [HIVE-18481] - Create tests for table related methods (get, list, exists)
+    * [HIVE-18483] - Create tests to cover getPartition(s) methods
+    * [HIVE-18484] - Create tests to cover listPartition(s) methods
+    * [HIVE-18486] - Create tests to cover add partition methods
+    * [HIVE-18489] - Automatically migrate s3n URIs to s3a URIs
+    * [HIVE-18495] - JUnit rule to enable Driver level testing
+    * [HIVE-18496] - Create tests to cover add/alter/drop index methods
+    * [HIVE-18498] - Create tests to cover get and list index methods
+    * [HIVE-18509] - Create tests for table manipulation related methods (create, alter, drop)
+    * [HIVE-18511] - Fix generated checkstyle errors
+    * [HIVE-18536] - IOW + DP is broken for insert-only ACID
+    * [HIVE-18541] - Secure HS2 web UI with PAM
+    * [HIVE-18542] - Create tests to cover getTableMeta method
+    * [HIVE-18544] - Create tests to cover appendPartition methods
+    * [HIVE-18550] - Keep the hbase table name property as hbase.table.name
+    * [HIVE-18553] - Support schema evolution in Parquet Vectorization reader
+    * [HIVE-18566] - Create tests to cover adding partitions from PartitionSpec
+    * [HIVE-18580] - Create tests to cover exchange partitions
+    * [HIVE-18596] - Synchronize value of hive.spark.client.connect.timeout across unit tests
+    * [HIVE-18609] - Results cache invalidation based on ACID table updates
+    * [HIVE-18633] - Service discovery for Active/Passive HA mode
+    * [HIVE-18635] - Generalize hook dispatch logics in Driver
+    * [HIVE-18651] - Expose additional Spark metrics
+    * [HIVE-18663] - Logged Spark Job Id contains a UUID instead of the actual id
+    * [HIVE-18672] - Printed state in RemoteSparkJobMonitor is ambiguous
+    * [HIVE-18673] - ErrorMsg.SPARK_JOB_MONITOR_TIMEOUT isn't formatted correctly
+    * [HIVE-18677] - SparkClientImpl usage of SessionState.LogHelper doesn't respect isSilent value
+    * [HIVE-18679] - create/replicate open transaction event
+    * [HIVE-18703] - Make Operator comparision to be based on some primitive
+    * [HIVE-18715] - Remove index support from metastore
+    * [HIVE-18720] - Replicate Commit Txn operation (without writes)
+    * [HIVE-18745] - Fix MetaStore creation in tests, so multiple MetaStores can be started on the same machine
+    * [HIVE-18747] - Cleaner for TXN_TO_WRITE_ID table entries using MIN_HISTORY_LEVEL.
+    * [HIVE-18749] - Need to replace transactionId with writeId in RecordIdentifier and other relevant contexts.
+    * [HIVE-18750] - Exchange partition should be disabled on ACID/Insert-only tables with per table write ID.
+    * [HIVE-18751] - ACID table scan through get_splits UDF doesn't receive ValidWriteIdList configuration.
+    * [HIVE-18753] - Correct methods and variables names which uses writeId instead of transactionId.
+    * [HIVE-18755] - Modifications to the metastore for catalogs
+    * [HIVE-18765] - SparkClientImpl swallows exception messages from the RemoteDriver
+    * [HIVE-18771] - Refactor tests, so only 1 MetaStore instance will be started per test class and test configuration
+    * [HIVE-18781] - Create/Replicate Open, Commit (without writes) and Abort Txn events
+    * [HIVE-18805] - Add ConstantPropagate before stats annotation
+    * [HIVE-18824] - ValidWriteIdList config should be defined on tables which has to collect stats after insert
+    * [HIVE-18830] - RemoteSparkJobMonitor failures are logged twice
+    * [HIVE-18832] - Support change management for trashing data files from ACID tables.
+    * [HIVE-18840] - CachedStore: Prioritize loading of recently accessed tables during prewarm
+    * [HIVE-18846] - Query results cache: Allow queries to refer to the pending results of a query that has not finished yet
+    * [HIVE-18855] - Fix unit test TestMiniLlapLocalCliDriver.testCliDriver[results_cache_1]
+    * [HIVE-18861] - druid-hdfs-storage is pulling in hadoop-aws-2.7.x and aws SDK, creating classpath problems on hadoop 3.x
+    * [HIVE-18864] - ValidWriteIdList snapshot seems incorrect if obtained after allocating writeId by current transaction.
+    * [HIVE-18899] - Separate FetchWork required for each query that uses the results cache
+    * [HIVE-18909] - Metrics for results cache
+    * [HIVE-18926] - Imporve operator-tree matching
+    * [HIVE-18946] - Fix columnstats merge NPE
+    * [HIVE-18961] - Error in results cache when query has identifiers with spaces
+    * [HIVE-18982] - Provide a CLI option to manually trigger failover
+    * [HIVE-18988] - Support bootstrap replication of ACID tables
+    * [HIVE-18994] - Handle client connections on failover
+    * [HIVE-19009] - Retain and use runtime statistics during hs2 lifetime
+    * [HIVE-19031] - Mark duplicate configs in HiveConf as deprecated
+    * [HIVE-19083] - Make partition clause optional for INSERT
+    * [HIVE-19089] - Create/Replicate Allocate write-id event
+    * [HIVE-19112] - Support Analyze table for partitioned tables without partition spec
+    * [HIVE-19126] - CachedStore: Use memory estimation to limit cache size during prewarm
+    * [HIVE-19127] - Concurrency fixes in QueryResultsCache
+    * [HIVE-19128] - Update golden files for spark perf tests
+    * [HIVE-19129] - Support DEFAULT keyword with MERGE
+    * [HIVE-19135] - Need tool to allow admins to create catalogs and move existing dbs to catalog during upgrade
+    * [HIVE-19138] - Results cache: allow queries waiting on pending cache entries to check cache again if pending query fails
+    * [HIVE-19141] - TestNegativeCliDriver insert_into_notnull_constraint, insert_into_acid_notnull failing
+    * [HIVE-19144] - TestSparkCliDriver:subquery_scalar - golden file needs to be udpated
+    * [HIVE-19145] - Stabilize statsoptimizer.q test
+    * [HIVE-19146] - Delete dangling q.out 
+    * [HIVE-19147] - Fix PerfCliDrivers: Tpcds30T missed CAT_NAME change
+    * [HIVE-19153] - Update golden files for few tests
+    * [HIVE-19154] - Poll notification events to invalidate the results cache
+    * [HIVE-19156] - TestMiniLlapLocalCliDriver.vectorized_dynamic_semijoin_reduction.q is broken
+    * [HIVE-19159] - TestMTQueries.testMTQueries1 failure
+    * [HIVE-19164] - TestMetastoreVersion failures
+    * [HIVE-19171] - Persist runtime statistics in metastore
+    * [HIVE-19175] - TestMiniLlapLocalCliDriver.testCliDriver update_access_time_non_current_db failing
+    * [HIVE-19178] - TestMiniTezCliDriver.testCliDriver[explainanalyze_5] failure
+    * [HIVE-19193] - TestActivePassiveHA fails
+    * [HIVE-19194] - TestDruidStorageHandler fails
+    * [HIVE-19195] - Fix flaky tests and cleanup testconfiguration to run llap specific tests in llap only.
+    * [HIVE-19196] - TestTriggersMoveWorkloadManager is flaky
+    * [HIVE-19197] - TestReplicationScenarios is flaky
+    * [HIVE-19206] - Automatic memory management for open streaming writers
+    * [HIVE-19209] - Streaming ingest record writers should accept input stream
+    * [HIVE-19210] - Create separate module for streaming ingest
+    * [HIVE-19211] - New streaming ingest API and support for dynamic partitioning
+    * [HIVE-19214] - High throughput ingest ORC format
+    * [HIVE-19222] - TestNegativeCliDriver tests are failing due to "java.lang.OutOfMemoryError: GC overhead limit exceeded"
+    * [HIVE-19232] - results_cache_invalidation2 is failing
+    * [HIVE-19274] - Add an OpTreeSignature persistence checker hook
+    * [HIVE-19319] - RuntimeStats fixes
+    * [HIVE-19322] - broken test: TestNegativeMinimrCliDriver#testCliDriver[minimr_broken_pipe]
+    * [HIVE-19335] - Disable runtime filtering (semijoin reduction opt with bloomfilter) for external tables
+    * [HIVE-19346] - TestMiniLlapLocalCliDriver.testCliDriver[materialized_view_create_rewrite_5] failling
+    * [HIVE-19347] - TestTriggersWorkloadManager tests are failing consistently
+    * [HIVE-19348] -  org.apache.hadoop.hive.ql.plan.mapping.TestOperatorCmp are failing
+    * [HIVE-19371] - Add table ownerType to HMS thrift API
+    * [HIVE-19372] - Add table ownerType to JDO/SQL and ObjectStore
+    * [HIVE-19374] - Parse and process ALTER TABLE SET OWNER command syntax
+    * [HIVE-19400] - Adjust Hive 1.0 to 2.0 conversion utility to the upgrade
+    * [HIVE-19471] - bucket_map_join_tez1 and  bucket_map_join_tez2 are failing
+    * [HIVE-19472] - HiveStreamingConnection swallows exception on partition creation
+    * [HIVE-19494] - Accept shade prefix during reflective instantiation of output format
+
+
+** Bug
+    * [HIVE-4577] - hive CLI can't handle hadoop dfs command  with space and quotes.
+    * [HIVE-6348] - Order by/Sort by in subquery
+    * [HIVE-6590] - Hive does not work properly with boolean partition columns (wrong results and inserts to incorrect HDFS path)
+    * [HIVE-6990] - Direct SQL fails when the explicit schema setting is different from the default one
+    * [HIVE-8937] - fix description of hive.security.authorization.sqlstd.confwhitelist.* params
+    * [HIVE-9815] - Metastore column"SERDE_PARAMS"."PARAM_VALUE"  limited to 4000 bytes
+    * [HIVE-10616] - TypeInfoUtils doesn't handle DECIMAL with just precision specified
+    * [HIVE-10865] - Beeline needs to support DELIMITER command
+    * [HIVE-11064] - ALTER TABLE CASCADE ERROR unbalanced calls to openTransaction/commitTransaction
+    * [HIVE-11266] - count(*) wrong result based on table statistics for external tables
+    * [HIVE-11297] - Combine op trees for partition info generating tasks
+    * [HIVE-11609] - Capability to add a filter to hbase scan via composite key doesn't work
+    * [HIVE-12408] - SQLStdAuthorizer should not require external table creator to be owner of directory, in addition to rw permissions
+    * [HIVE-12425] - OrcRecordUpdater.close(true) leaves the file open
+    * [HIVE-12631] - LLAP IO: support ORC ACID tables
+    * [HIVE-12719] - As a hive user, I am facing issues using permanent UDAF's.
+    * [HIVE-12734] - Remove redundancy in HiveConfs serialized to UDFContext
+    * [HIVE-13000] - Hive returns useless parsing error 
+    * [HIVE-13652] - Import table change order of dynamic partitions
+    * [HIVE-14032] - INSERT OVERWRITE command failed with case sensitive partition key names
+    * [HIVE-14052] - Cleanup structures when external clients use LLAP
+    * [HIVE-14077] - add implicit decimal arithmetic q test, fix issues if found 
+    * [HIVE-14455] - upgrade httpclient, httpcore to match updated hadoop dependency
+    * [HIVE-14560] - Support exchange partition between s3 and hdfs tables
+    * [HIVE-14564] - Column Pruning generates out of order columns in SelectOperator which cause ArrayIndexOutOfBoundsException.
+    * [HIVE-14678] - Hive-on-MR deprecation warning  is not diplayed when engine is set to capital letter 'MR'
+    * [HIVE-14731] - Use Tez cartesian product edge in Hive (unpartitioned case only)
+    * [HIVE-14792] - AvroSerde reads the remote schema-file at least once per mapper, per table reference.
+    * [HIVE-14813] - Make TransactionBatchImpl.toString() include state of each txn: commit/abort
+    * [HIVE-14988] - Support INSERT OVERWRITE into a partition on transactional tables
+    * [HIVE-15077] - Acid LockManager is unfair
+    * [HIVE-15104] - Hive on Spark generate more shuffle data than hive on mr
+    * [HIVE-15144] - JSON.org license is now CatX
+    * [HIVE-15160] - Can't order by an unselected column
+    * [HIVE-15176] - Small typo in hiveserver2 webui
+    * [HIVE-15249] - HIve 2.1.0 is throwing InvalidObjectException(message:Invalid column type name is too long
+    * [HIVE-15267] - Make query length calculation logic more accurate in TxnUtils.needNewQuery()
+    * [HIVE-15343] - Spelling errors in logging and exceptions for beeline, common, hbase-handler, hcatalog, llap-server, orc, serde and shims
+    * [HIVE-15344] - Spelling errors in logging and exceptions for metastore and service directories
+    * [HIVE-15442] - Driver.java has a redundancy  code
+    * [HIVE-15483] - Database and table name is case sensitive when used in show grant
+    * [HIVE-15504] - ArrayIndexOutOfBoundsException in GenericUDFTrunc::initialize
+    * [HIVE-15515] - Remove the docs directory
+    * [HIVE-15552] - Unable to coalesce DATE and TIMESTAMP types
+    * [HIVE-15630] - add operation handle before operation.run instead of after operation.run
+    * [HIVE-15632] - Hive/Druid integration: Incorrect result - Limit on timestamp disappears
+    * [HIVE-15635] - Hive/Druid integration: timeseries query shows all days, even if no data
+    * [HIVE-15636] - Hive/Druid integration: wrong semantics of topN query limit with granularity
+    * [HIVE-15637] - Hive/Druid integration: wrong semantics of groupBy query limit with granularity
+    * [HIVE-15639] - Hive/Druid integration: wrong semantics for ordering within groupBy queries
+    * [HIVE-15680] - Incorrect results when hive.optimize.index.filter=true and same ORC table is referenced twice in query
+    * [HIVE-15724] - getPrimaryKeys and getForeignKeys in metastore does not normalize db and table name
+    * [HIVE-15739] - Incorrect exception message in PartExprEvalUtils
+    * [HIVE-15761] - ObjectStore.getNextNotification could return an empty NotificationEventResponse causing TProtocolException 
+    * [HIVE-15767] - Hive On Spark is not working on secure clusters from Oozie
+    * [HIVE-15829] - LLAP text cache: disable memory tracking on the writer
+    * [HIVE-15883] - HBase mapped table in Hive insert fail for decimal
+    * [HIVE-15995] - Syncing metastore table with serde schema
+    * [HIVE-16007] - When the query does not complie the LogRunnable never stops
+    * [HIVE-16025] - Where IN clause throws exception
+    * [HIVE-16026] - Generated query will timeout and/or kill the druid cluster.
+    * [HIVE-16027] - <timestamp> BETWEEN <string> AND <string> must cast to TIMESTMAP
+    * [HIVE-16044] - LLAP: Shuffle Handler keep-alive connections are closed from the server side
+    * [HIVE-16053] - Remove newRatio from llap JAVA_OPTS_BASE
+    * [HIVE-16057] - SchemaTool ignores --passWord argument if hadoop.security.credential.provider.path is configured
+    * [HIVE-16061] - When hive.async.log.enabled is set to true, some output is not printed to the beeline console
+    * [HIVE-16077] - UPDATE/DELETE fails with numBuckets > numReducers
+    * [HIVE-16113] - PartitionPruner::removeNonPartCols needs to handle AND/OR cases
+    * [HIVE-16117] - SortProjectTransposeRule should check for monotonicity preserving CAST
+    * [HIVE-16125] - Split work between reducers.
+    * [HIVE-16130] - Remove jackson classes from hive-jdbc standalone jar
+    * [HIVE-16147] - Rename a partitioned table should not drop its partition columns stats
+    * [HIVE-16174] - Update MetricsConstant.WAITING_COMPILE_OPS metric when we aquire lock failed in Driver
+    * [HIVE-16177] - non Acid to acid conversion doesn't handle _copy_N files
+    * [HIVE-16188] - beeline should block the connection if given invalid database name.
+    * [HIVE-16193] - Hive show compactions not reflecting the status of the application
+    * [HIVE-16213] - ObjectStore can leak Queries when rollbackTransaction throws an exception
+    * [HIVE-16219] - metastore notification_log contains serialized message with  non functional fields
+    * [HIVE-16222] - add a setting to disable row.serde for specific formats; enable for others
+    * [HIVE-16225] - Memory leak in webhcat service (FileSystem CACHE entries)
+    * [HIVE-16233] - llap: Query failed with AllocatorOutOfMemoryException
+    * [HIVE-16254] - metadata for values temporary tables for INSERTs are getting replicated during bootstrap
+    * [HIVE-16275] - Vectorization: Add ReduceSink support for TopN (in specialized native classes)
+    * [HIVE-16282] - Semijoin: Disable slow-start for the bloom filter aggregate task
+    * [HIVE-16287] - Alter table partition rename with location - moves partition back to hive warehouse
+    * [HIVE-16290] - Stats: StatsRulesProcFactory::evaluateComparator estimates are wrong when minValue == filterValue
+    * [HIVE-16291] - Hive fails when unions a parquet table with itself
+    * [HIVE-16296] - use LLAP executor count to configure reducer auto-parallelism
+    * [HIVE-16298] - Add config to specify multi-column joins have correlated columns
+    * [HIVE-16299] - MSCK REPAIR TABLE should enforce partition key order when adding unknown partitions
+    * [HIVE-16302] - Add junit dependency to hive-shims-common to compile with Hadoop 2.8+
+    * [HIVE-16305] - Additional Datanucleus ClassLoaderResolverImpl leaks causing HS2 OOM
+    * [HIVE-16307] - add IO memory usage report to LLAP UI
+    * [HIVE-16308] - PreExecutePrinter and PostExecutePrinter should log to INFO level instead of ERROR
+    * [HIVE-16309] - Hive Test Commands failure should be printed in hive.log in addition to stderr
+    * [HIVE-16315] - Describe table doesn't show num of partitions
+    * [HIVE-16316] - Prepare master branch for 3.0.0 development.
+    * [HIVE-16317] - CASE .. NULL in JOIN condition can trigger SemanticException
+    * [HIVE-16318] - LLAP cache: address some issues in 2.2/2.3
+    * [HIVE-16319] - LLAP: Better handling of an empty wait queue, should try scheduling checks
+    * [HIVE-16321] - Possible deadlock in metastore with Acid enabled
+    * [HIVE-16323] - HS2 JDOPersistenceManagerFactory.pmCache leaks after HIVE-14204
+    * [HIVE-16324] - Truncate table should not work when EXTERNAL property of table is true
+    * [HIVE-16325] - Tez session refresh based on a time interval fails
+    * [HIVE-16328] - HoS: more aggressive mapjoin optimization when hive.spark.use.ts.stats.for.mapjoin is true
+    * [HIVE-16329] - TopN: use local executor info for LLAP memory checks
+    * [HIVE-16333] - remove the redundant symbol "\" to appear red in sublime text 3
+    * [HIVE-16335] - Beeline user HS2 connection file should use /etc/hive/conf instead of /etc/conf/hive
+    * [HIVE-16336] - Rename hive.spark.use.file.size.for.mapjoin to hive.spark.use.ts.stats.for.mapjoin
+    * [HIVE-16341] - Tez Task Execution Summary has incorrect input record counts on some operators
+    * [HIVE-16347] - HiveMetastoreChecker should skip listing partitions which are not valid when hive.msck.path.validation is set to skip or ignore
+    * [HIVE-16353] - Jetty 9 upgrade breaks hive master LLAP
+    * [HIVE-16357] - Failed folder creation when creating a new table is reported incorrectly
+    * [HIVE-16363] - QueryLifeTimeHooks should catch parse exceptions
+    * [HIVE-16368] - Unexpected java.lang.ArrayIndexOutOfBoundsException from query with LaterView Operation for hive on MR.
+    * [HIVE-16369] - Vectorization: Support PTF (Part 1: No Custom Window Framing -- Default Only)
+    * [HIVE-16380] - removing global test dependency of jsonassert
+    * [HIVE-16384] - Remove jdk7 build from travis
+    * [HIVE-16385] - StatsNoJobTask could exit early before all partitions have been processed
+    * [HIVE-16388] - LLAP: Log rotation for daemon, history and gc files
+    * [HIVE-16389] - Allow HookContext to access SQLOperationDisplay
+    * [HIVE-16390] - LLAP IO should take job config into account; also LLAP config should load defaults
+    * [HIVE-16393] - Fix visibility of CodahaleReporter interface
+    * [HIVE-16394] - HoS does not support queue name change in middle of session
+    * [HIVE-16396] - Sync storage-api version in pom.xml
+    * [HIVE-16399] - create an index for tc_txnid in TXN_COMPONENTS
+    * [HIVE-16402] - Upgrade to Hadoop 2.8.0
+    * [HIVE-16403] - LLAP UI shows the wrong number of executors
+    * [HIVE-16404] - Renaming of public classes in Calcite 12 breeaking druid integration
+    * [HIVE-16406] - Remove unwanted interning when creating PartitionDesc
+    * [HIVE-16409] - TestEventHandlerFactory  has lacked the ASF header
+    * [HIVE-16413] - Create table as select does not check ownership of the location
+    * [HIVE-16421] - Runtime filtering breaks user-level explain
+    * [HIVE-16422] - Should kill running Spark Jobs when a query is cancelled.
+    * [HIVE-16425] - Vectorization: unload old hashtables before reloadHashTable
+    * [HIVE-16427] - Fix multi-insert query and write qtests
+    * [HIVE-16433] - Not nullify variable "rj" to avoid NPE due to race condition in ExecDriver.
+    * [HIVE-16436] - Response times in "Task Execution Summary" at the end of the job is not correct
+    * [HIVE-16448] - Vectorization: Vectorized order_null.q fails with deserialize EOF exception below TEZ ReduceRecordSource.processVectorGroup
+    * [HIVE-16450] - Some metastore operations are not retried even with desired underlining exceptions
+    * [HIVE-16451] - Race condition between HiveStatement.getQueryLog and HiveStatement.runAsyncOnServer
+    * [HIVE-16459] - Forward channelInactive to RpcDispatcher
+    * [HIVE-16461] - DagUtils checks local resource size on the remote fs
+    * [HIVE-16462] - Vectorization: Enabling hybrid grace disables specialization of all reduce side joins
+    * [HIVE-16465] - NullPointer Exception when enable vectorization for Parquet file format
+    * [HIVE-16468] - BeeLineDriver should be able to run tests against an externally created cluster
+    * [HIVE-16471] - Add metrics for "waiting compilation time"
+    * [HIVE-16473] - Hive-on-Tez may fail to write to an HBase table
+    * [HIVE-16482] - Druid Ser/Des need to use dimension output name
+    * [HIVE-16483] - HoS should populate split related configurations to HiveConf
+    * [HIVE-16485] - Enable outputName for RS operator in explain formatted
+    * [HIVE-16487] - Serious Zookeeper exception is logged when a race condition happens
+    * [HIVE-16491] - CBO cant handle join involving complex types in on condition
+    * [HIVE-16494] - udaf percentile_approx() may fail on CBO
+    * [HIVE-16497] - FileUtils. isActionPermittedForFileHierarchy, isOwnerOfFileHierarchy file system operations should be impersonated
+    * [HIVE-16507] - Hive Explain User-Level may print out "Vertex dependency in root stage" twice
+    * [HIVE-16510] - Vectorization: Add vectorized PTF tests in preparation for HIVE-16369
+    * [HIVE-16511] - CBO looses inner casts on constants of complex type
+    * [HIVE-16513] - width_bucket issues
+    * [HIVE-16518] - Insert override for druid does not replace all existing segments
+    * [HIVE-16519] - Fix exception thrown by checkOutputSpecs
+    * [HIVE-16523] - VectorHashKeyWrapper hash code for strings is not so good
+    * [HIVE-16524] - Remove the redundant item type in hiveserver2.jsp and QueryProfileTmpl.jamon
+    * [HIVE-16533] - Vectorization: Avoid evaluating empty groupby keys
+    * [HIVE-16534] - Add capability to tell aborted transactions apart from open transactions in ValidTxnList
+    * [HIVE-16538] - TestExecDriver fails if run after TestOperators#testScriptOperator
+    * [HIVE-16539] - Add PTF tests for blobstores
+    * [HIVE-16545] - LLAP: bug in arena size determination logic
+    * [HIVE-16546] - LLAP: Fail map join tasks if hash table memory exceeds threshold
+    * [HIVE-16547] - LLAP: may not unlock buffers in some cases
+    * [HIVE-16553] - Change default value for hive.tez.bigtable.minsize.semijoin.reduction
+    * [HIVE-16554] - ACID: Make HouseKeeperService threads daemon
+    * [HIVE-16557] - Vectorization: Specialize ReduceSink empty key case
+    * [HIVE-16559] - Parquet schema evolution for partitioned tables may break if table and partition serdes differ
+    * [HIVE-16562] - Issues with nullif / fetch task
+    * [HIVE-16563] - Alter table partition set location should use fully qualified path for non-default FS
+    * [HIVE-16572] - Rename a partition should not drop its column stats
+    * [HIVE-16573] - In-place update for HoS can't be disabled
+    * [HIVE-16576] - Fix encoding of intervals when fetching select query candidates from druid
+    * [HIVE-16577] - Syntax error in the metastore init scripts for mssql
+    * [HIVE-16578] - Semijoin Hints should use column name, if provided for partition key check
+    * [HIVE-16581] -  a bug in HIVE-16523
+    * [HIVE-16584] - Warning messages should use LogHelper.printInfo instead of printing to the infoStream directly
+    * [HIVE-16588] - Resource leak by druid http client
+    * [HIVE-16589] - Vectorization: Support Complex Types and GroupBy modes PARTIAL2, FINAL, and COMPLETE  for AVG, VARIANCE
+    * [HIVE-16590] - Make initializing dag names in SparkWork thread safe for parallel compilation (HIVE-13512)
+    * [HIVE-16592] - Vectorization: Long hashCodes should bit-mix into lower bits
+    * [HIVE-16593] - SparkClientFactory.stop may prevent JVM from exiting
+    * [HIVE-16598] - LlapServiceDriver - create directories and warn of errors
+    * [HIVE-16599] - NPE in runtime filtering cost when handling SMB Joins
+    * [HIVE-16603] - Enforce foreign keys to refer to primary keys or unique keys
+    * [HIVE-16607] - ColumnStatsAutoGatherContext regenerates HiveConf.HIVEQUERYID
+    * [HIVE-16609] - col='__HIVE_DEFAULT_PARTITION__' condition in select statement may produce wrong result
+    * [HIVE-16610] - Semijoin Hint : Should be able to handle more than one hint per alias
+    * [HIVE-16613] - SaslClientHandler.sendHello is eating exceptions
+    * [HIVE-16625] - Extra '\0' characters in the output, when SeparatedValuesOutputFormat is used and the quoting is disabled
+    * [HIVE-16633] - username for ATS data shall always be the uid who submit the job
+    * [HIVE-16634] - LLAP Use a pool of connections to a single AM from a daemon
+    * [HIVE-16640] - The ASF Headers have some errors in some class
+    * [HIVE-16645] - Commands.java has missed the catch statement and has some code format errors
+    * [HIVE-16646] - Alias in transform ... as clause shouldn't be case sensitive
+    * [HIVE-16654] - Optimize a combination of avg(), sum(), count(distinct) etc
+    * [HIVE-16658] - TestTimestampTZ.java has missed the ASF header
+    * [HIVE-16659] - Query plan should reflect hive.spark.use.groupby.shuffle
+    * [HIVE-16660] - Not able to add partition for views in hive when sentry is enabled
+    * [HIVE-16665] - Race condition in Utilities.GetInputPathsCallable --> createDummyFileForEmptyPartition
+    * [HIVE-16667] - PostgreSQL metastore handling of CLOB types for COLUMNS_V2.TYPE_NAME and other field is incorrect
+    * [HIVE-16671] - LLAP IO: BufferUnderflowException may happen in very rare(?) cases due to ORC end-of-CB estimation
+    * [HIVE-16675] - Fix ConcurrentModificationException in SparkClientImpl#startDriver
+    * [HIVE-16677] - CTAS with no data fails in Druid
+    * [HIVE-16678] - Truncate on temporary table fails with "table not found" error.
+    * [HIVE-16679] - Missing ASF header on properties file in ptest2 project
+    * [HIVE-16689] - Correlated scalar subquery with comparison to constant in predicate fails
+    * [HIVE-16692] - LLAP: Keep alive connection in shuffle handler should not be closed until entire data is flushed out
+    * [HIVE-16693] - beeline "source" command freezes if you have a comment in it?
+    * [HIVE-16696] - Fix JoinCondDesc explain string
+    * [HIVE-16698] - HoS should avoid mapjoin optimization in case of union and using table stats
+    * [HIVE-16703] - Hive may add the same file to the session and vertex in Tez
+    * [HIVE-16708] - Exception while renewing a Delegation Token
+    * [HIVE-16721] - Inconsistent behavior in dealing with Timestamp stats
+    * [HIVE-16724] - increase session timeout for LLAP ZK token manager
+    * [HIVE-16730] - Vectorization: Schema Evolution for Text Vectorization / Complex Types
+    * [HIVE-16731] - Vectorization: Make "CASE WHEN (day_name='Sunday') THEN column1 ELSE null end" that involves a column name or expression THEN or ELSE vectorize
+    * [HIVE-16732] - Transactional tables should block LOAD DATA 
+    * [HIVE-16737] - LLAP: Shuffle handler TCP listen queue overflows
+    * [HIVE-16738] - Notification ID generation in DBNotification might not be unique across HS2 instances.
+    * [HIVE-16742] - cap the number of reducers for LLAP at the configured value
+    * [HIVE-16743] - BitSet set() is incorrectly used in TxnUtils.createValidCompactTxnList()
+    * [HIVE-16744] - LLAP index update may be broken after ORC switch
+    * [HIVE-16745] - Syntax error in 041-HIVE-16556.mysql.sql script
+    * [HIVE-16746] - Reduce number of index lookups for same table in IndexWhereTaskDispatcher
+    * [HIVE-16751] - Support different types for grouping columns in GroupBy Druid queries
+    * [HIVE-16755] - LLAP IO: incorrect assert may trigger in tests
+    * [HIVE-16756] - Vectorization: LongColModuloLongColumn throws "java.lang.ArithmeticException: / by zero"
+    * [HIVE-16757] - Use of deprecated getRows() instead of new estimateRowCount(RelMetadataQuery..) has serious performance impact
+    * [HIVE-16761] - LLAP IO: SMB joins fail elevator 
+    * [HIVE-16769] - Possible hive service startup due to the existing file /tmp/stderr
+    * [HIVE-16776] - Strange cast behavior for table backed by druid
+    * [HIVE-16777] - LLAP: Use separate tokens and UGI instances when an external client is used
+    * [HIVE-16778] - LLAP IO: better refcount management
+    * [HIVE-16780] - Case "multiple sources, single key" in spark_dynamic_pruning.q fails 
+    * [HIVE-16784] - Missing lineage information when hive.blobstore.optimizations.enabled is true
+    * [HIVE-16788] - ODBC call SQLForeignKeys leads to NPE if you use PK arguments rather than FK arguments
+    * [HIVE-16793] - Scalar sub-query: sq_count_check not required if gby keys are constant
+    * [HIVE-16801] - Vectorization: throwExpandError should be an immediate fatal 
+    * [HIVE-16803] - Alter table change column comment should not try to get column stats for update
+    * [HIVE-16804] - Semijoin hint : Needs support for target table.
+    * [HIVE-16808] - WebHCat statusdir parameter doesn't properly handle Unicode characters when using relative path
+    * [HIVE-16820] - TezTask may not shut down correctly before submit
+    * [HIVE-16821] - Vectorization: support Explain Analyze in vectorized mode
+    * [HIVE-16824] - PrimaryToReplicaResourceFunctionTest.java has missed the ASF header
+    * [HIVE-16826] - Improvements for SeparatedValuesOutputFormat
+    * [HIVE-16828] - With CBO enabled, Query on partitioned views throws IndexOutOfBoundException
+    * [HIVE-16832] - duplicate ROW__ID possible in multi insert into transactional table
+    * [HIVE-16835] - Addendum to HIVE-16745
+    * [HIVE-16844] - Fix Connection leak in ObjectStore when new Conf object is used
+    * [HIVE-16845] - INSERT OVERWRITE a table with dynamic partitions on S3 fails with NPE
+    * [HIVE-16846] - TestJdbcWithMiniHS2#testHttpHeaderSize test case is not testing in HTTP mode
+    * [HIVE-16847] - LLAP queue order issue
+    * [HIVE-16851] - Scalar subquery with group by missing sq_count_check UDF
+    * [HIVE-16854] - SparkClientFactory is locked too aggressively
+    * [HIVE-16864] - add validation to stream position search in LLAP IO
+    * [HIVE-16869] - Hive returns wrong result when predicates on non-existing columns are pushed down to Parquet reader
+    * [HIVE-16871] - CachedStore.get_aggr_stats_for has side affect
+    * [HIVE-16875] - Query against view with partitioned child on HoS fails with privilege exception.
+    * [HIVE-16876] - HoS: Make Rpc configs immutable at runtime
+    * [HIVE-16877] - NPE when issue query like alter table ... cascade onto non-partitioned table 
+    * [HIVE-16886] - HMS log notifications may have duplicated event IDs if multiple HMS are running concurrently
+    * [HIVE-16888] - Upgrade Calcite to 1.13 and Avatica to 1.10
+    * [HIVE-16898] - Validation of source file after distcp in repl load 
+    * [HIVE-16902] - investigate "failed to remove operation log" errors
+    * [HIVE-16903] - LLAP: Fix config name issue in SHUFFLE_MANAGE_OS_CACHE
+    * [HIVE-16908] - Failures in TestHcatClient due to HIVE-16844
+    * [HIVE-16910] - RpcConfiguration - Improper Cast From Long To Int
+    * [HIVE-16915] - partition column count is not determined correctly in LLAP IO non-vectorized wrapper
+    * [HIVE-16918] - Skip ReplCopyTask distcp for _metadata copying. Also enable -pb for distcp
+    * [HIVE-16920] - remove useless uri.getScheme() from EximUtil
+    * [HIVE-16922] - Typo in serde.thrift: COLLECTION_DELIM = "colelction.delim"
+    * [HIVE-16927] - LLAP: Slider takes down all daemons when some daemons fail repeatedly
+    * [HIVE-16930] - HoS should verify the value of Kerberos principal and keytab file before adding them to spark-submit command parameters
+    * [HIVE-16935] - Hive should strip comments from input before choosing which CommandProcessor to run.
+    * [HIVE-16937] - INFORMATION_SCHEMA usability: everything is currently a string
+    * [HIVE-16938] - INFORMATION_SCHEMA usability: difficult to access # of table records
+    * [HIVE-16939] - metastore error: 'export: -Dproc_metastore : not a valid identifier'
+    * [HIVE-16942] - INFORMATION_SCHEMA: schematool for setting it up is not idempotent
+    * [HIVE-16943] - MoveTask should separate src FileSystem from dest FileSystem 
+    * [HIVE-16947] - Semijoin Reduction : Task cycle created due to multiple semijoins in conjunction with hashjoin
+    * [HIVE-16948] - Invalid explain when running dynamic partition pruning query in Hive On Spark
+    * [HIVE-16949] - Leak of threads from Get-Input-Paths and Get-Input-Summary thread pool
+    * [HIVE-16954] - LLAP IO: better debugging
+    * [HIVE-16958] - Setting hive.merge.sparkfiles=true will retrun an error when generating parquet databases 
+    * [HIVE-16960] - Hive throws an ugly error exception when HDFS sticky bit is set
+    * [HIVE-16961] - Hive on Spark leaks spark application in case user cancels query and closes session
+    * [HIVE-16964] - _orc_acid_version file is missing
+    * [HIVE-16965] - SMB join may produce incorrect results
+    * [HIVE-16973] - Fetching of Delegation tokens (Kerberos) for AccumuloStorageHandler fails in HS2
+    * [HIVE-16975] - Vectorization: Fully vectorize CAST date as TIMESTAMP so VectorUDFAdaptor is now used
+    * [HIVE-16978] - HoS: add current thread ID to the log redirector for the RemoteDriver
+    * [HIVE-16982] - WebUI "Show Query" tab prints "UNKNOWN" instead of explaining configuration option
+    * [HIVE-16985] - LLAP IO: enable SMB join in elevator after the former is fixed
+    * [HIVE-16991] - HiveMetaStoreClient needs a 2-arg constructor for backwards compatibility
+    * [HIVE-17002] - decimal (binary) is not working when creating external table for hbase
+    * [HIVE-17006] - LLAP: Parquet caching v1
+    * [HIVE-17007] - NPE introduced by HIVE-16871
+    * [HIVE-17008] - Fix boolean flag switchup in DropTableEvent
+    * [HIVE-17010] - Fix the overflow problem of Long type in SetSparkReducerParallelism
+    * [HIVE-17013] - Delete request with a subquery based on select over a view
+    * [HIVE-17050] - Multiline queries that have comment in middle fail when executed via "beeline -e"
+    * [HIVE-17052] - Remove logging of predicate filters
+    * [HIVE-17066] - Query78 filter wrong estimatation is generating bad plan
+    * [HIVE-17067] - LLAP: Add http endpoint to provide system level configurations
+    * [HIVE-17069] - Refactor OrcRawRecrodMerger.ReaderPair
+    * [HIVE-17070] - remove .orig files from src
+    * [HIVE-17073] - Incorrect result with vectorization and SharedWorkOptimizer
+    * [HIVE-17076] - typo in itests/src/test/resources/testconfiguration.properties
+    * [HIVE-17079] - LLAP: Use FQDN by default for work submission
+    * [HIVE-17083] - DagUtils overwrites any credentials already added
+    * [HIVE-17085] - ORC file merge/concatenation should do full schema check
+    * [HIVE-17086] - LLAP: JMX Metric for max file descriptors used so far
+    * [HIVE-17088] - HS2 WebUI throws a NullPointerException when opened
+    * [HIVE-17090] - spark.only.query.files are not being run by ptest
+    * [HIVE-17093] - LLAP ssl configs need to be localized to talk to a wire encrypted hdfs
+    * [HIVE-17095] - Long chain repl loads do not complete in a timely fashion
+    * [HIVE-17097] - Fix SemiJoinHint parsing in SemanticAnalyzer
+    * [HIVE-17098] - Race condition in Hbase tables
+    * [HIVE-17099] - Update golden files for spark.only.query.files
+    * [HIVE-17109] - Remove calls to RelMetadataQuery.instance() after Calcite 1.13 upgrade
+    * [HIVE-17110] - BucketCodec should enforce value ranges
+    * [HIVE-17111] - Add TestLocalSparkCliDriver
+    * [HIVE-17113] - Duplicate bucket files can get written to table by runaway task
+    * [HIVE-17114] - HoS: Possible skew in shuffling when data is not really skewed
+    * [HIVE-17115] - MetaStoreUtils.getDeserializer doesn't catch the java.lang.ClassNotFoundException
+    * [HIVE-17116] - Vectorization: Add infrastructure for vectorization of ROW__ID struct
+    * [HIVE-17117] - Metalisteners are not notified when threadlocal metaconf is cleanup 
+    * [HIVE-17128] - Operation Logging leaks file descriptors as the log4j Appender is never closed
+    * [HIVE-17144] - export of temporary tables not working and it seems to be using distcp rather than filesystem copy
+    * [HIVE-17147] - Vectorization: Add code for testing MapJoin operator in isolation and measuring its performance with JMH
+    * [HIVE-17148] - Incorrect result for Hive join query with COALESCE in WHERE condition
+    * [HIVE-17149] - Hdfs directory is not cleared if partition creation failed on HMS
+    * [HIVE-17150] - CREATE INDEX execute HMS out-of-transaction listener calls inside a transaction
+    * [HIVE-17152] - Improve security of random generator for HS2 cookies
+    * [HIVE-17155] - findConfFile() in HiveConf.java has some issues with the conf path
+    * [HIVE-17169] - Avoid extra call to KeyProvider::getMetadata()
+    * [HIVE-17172] - add ordering checks to DiskRangeList
+    * [HIVE-17176] - Add ASF header for LlapAllocatorBuffer.java
+    * [HIVE-17177] - move TestSuite.java to the right position
+    * [HIVE-17181] - HCatOutputFormat should expose complete output-schema (including partition-keys) for dynamic-partitioning MR jobs
+    * [HIVE-17184] - Unexpected new line in beeline output when running with -f option
+    * [HIVE-17188] - ObjectStore runs out of memory for large batches of addPartitions().
+    * [HIVE-17189] - Fix backwards incompatibility in HiveMetaStoreClient
+    * [HIVE-17208] - Repl dump should pass in db/table information to authorization API
+    * [HIVE-17209] - ObjectCacheFactory should return null when tez shared object registry is not setup
+    * [HIVE-17213] - HoS: file merging doesn't work for union all
+    * [HIVE-17217] - SMB Join : Assert if paths are different in TezGroupedSplit in KeyValueInputMerger
+    * [HIVE-17218] - Canonical-ize hostnames for Hive metastore, and HS2 servers.
+    * [HIVE-17220] - Bloomfilter probing in semijoin reduction is thrashing L1 dcache
+    * [HIVE-17222] - Llap: Iotrace throws  java.lang.UnsupportedOperationException with IncompleteCb
+    * [HIVE-17228] - Bump tez version to 0.9.0
+    * [HIVE-17233] - Set "mapred.input.dir.recursive" for HCatInputFormat-based jobs.
+    * [HIVE-17235] - Add ORC Decimal64 Serialization/Deserialization (Part 1)
+    * [HIVE-17240] - Function ACOS(2) and ASIN(2) should be null
+    * [HIVE-17254] - Skip updating AccessTime of recycled files in ReplChangeManager
+    * [HIVE-17257] - Hive should merge empty files
+    * [HIVE-17258] - Incorrect log messages in the Hive.java
+    * [HIVE-17259] - Hive JDBC does not recognize UNIONTYPE columns
+    * [HIVE-17260] - Typo: exception has been created and lost in the ThriftJDBCBinarySerDe
+    * [HIVE-17265] - Cache merged column stats from retrieved partitions
+    * [HIVE-17267] - Make HMS Notification Listeners typesafe
+    * [HIVE-17268] - WebUI / QueryPlan: query plan is sometimes null when explain output conf is on
+    * [HIVE-17270] - Qtest results show wrong number of executors
+    * [HIVE-17272] - when hive.vectorized.execution.enabled is true, query on empty partitioned table fails with NPE
+    * [HIVE-17274] - RowContainer spills for timestamp column throws exception
+    * [HIVE-17275] - Auto-merge fails on writes of UNION ALL output to ORC file with dynamic partitioning
+    * [HIVE-17276] - Check max shuffle size when converting to dynamically partitioned hash join
+    * [HIVE-17277] - HiveMetastoreClient Log name is wrong
+    * [HIVE-17280] - Data loss in CONCATENATE ORC created by Spark
+    * [HIVE-17281] - LLAP external client not properly handling KILLED notification that occurs when a fragment is rejected
+    * [HIVE-17283] - Enable parallel edges of semijoin along with mapjoins
+    * [HIVE-17285] - Fixes for bit vector retrievals and merging
+    * [HIVE-17286] - Avoid expensive String serialization/deserialization for bitvectors
+    * [HIVE-17290] - Should use equals() rather than == to compare strings
+    * [HIVE-17298] - export when running distcp for large number of files should not run as privileged user 
+    * [HIVE-17301] - Make JSONMessageFactory.getTObj method thread safe
+    * [HIVE-17302] - ReduceRecordSource should not add batch string to Exception message
+    * [HIVE-17303] - Missmatch between roaring bitmap library used by druid and the one coming from tez
+    * [HIVE-17305] - New insert overwrite dynamic partitions qtest need to have the golden file regenerated
+    * [HIVE-17309] - alter partition onto a table not in current database throw InvalidOperationException
+    * [HIVE-17311] - Numeric overflow in the HiveConf
+    * [HIVE-17313] - Potentially possible 'case fall through' in the ObjectInspectorConverters
+    * [HIVE-17314] - LazySimpleSerializeWrite.writeString() contains if with an empty body
+    * [HIVE-17321] - HoS: analyze ORC table doesn't compute raw data size when noscan/partialscan is not specified
+    * [HIVE-17322] - Serialise BeeLine qtest execution to prevent flakyness
+    * [HIVE-17327] - LLAP IO: restrict native file ID usage to default FS to avoid hypothetical collisions when HDFS federation is used
+    * [HIVE-17331] - Path must be used as key type of the pathToAlises
+    * [HIVE-17333] - Schema changes in HIVE-12274 for Oracle may not work for upgrade
+    * [HIVE-17336] - Missing class 'org.apache.hadoop.hive.hbase.HiveHBaseTableInputFormat' from Hive on Spark when inserting into hbase based table
+    * [HIVE-17338] - Utilities.get*Tasks multiple methods duplicate code
+    * [HIVE-17344] - LocalCache element memory usage is not calculated properly.
+    * [HIVE-17348] - Remove unnecessary GenSparkUtils.java.orig file
+    * [HIVE-17351] - use new slider package installation command in run.sh
+    * [HIVE-17352] - HiveSever2 error with "Illegal Operation state transition from CLOSED to FINISHED"
+    * [HIVE-17354] - Fix "alter view" for incremental replication
+    * [HIVE-17356] - Missing ASF headers 3 classes
+    * [HIVE-17357] - Plugin jars are not properly added for LocalHiveSparkClient
+    * [HIVE-17360] - Tez session reopen appears to use a wrong conf object
+    * [HIVE-17364] - Add unit test to "alter view" replication
+    * [HIVE-17365] - Druid CTAS should support CHAR/VARCHAR type
+    * [HIVE-17367] - IMPORT table doesn't load from data dump if a metadata-only dump was already imported.
+    * [HIVE-17368] - DBTokenStore fails to connect in Kerberos enabled remote HMS environment
+    * [HIVE-17372] - update druid dependency to druid 0.10.1
+    * [HIVE-17377] - SharedWorkOptimizer might not iterate through TS operators deterministically
+    * [HIVE-17378] - CBO: HiveReduceExpressionsWithStatsRule can operate on IS_NULL and IS_NOT_NULL
+    * [HIVE-17385] - Fix incremental repl error for non-native tables
+    * [HIVE-17389] - Yetus is always failing on rat checks
+    * [HIVE-17391] - Compaction fails if there is an empty value in tblproperties
+    * [HIVE-17392] - SharedWorkOptimizer might merge TS operators filtered by not equivalent semijoin operators
+    * [HIVE-17393] - AMReporter need hearbeat every external 'AM'
+    * [HIVE-17394] - AvroSerde is regenerating TypeInfo objects for each nullable Avro field for every row
+    * [HIVE-17401] - Hive session idle timeout doesn't function properly
+    * [HIVE-17403] - Fail concatenation for unmanaged and transactional tables
+    * [HIVE-17410] - repl load task during subsequent DAG generation does not start from the last partition processed
+    * [HIVE-17411] - LLAP IO may incorrectly release a refcount in some rare cases
+    * [HIVE-17412] - Add "-- SORT_QUERY_RESULTS" for spark_vectorized_dynamic_partition_pruning.q
+    * [HIVE-17413] - predicate involving CAST affects value returned by the SELECT statement
+    * [HIVE-17415] - Hit error "SemanticException View xxx is corresponding to LIMIT, rather than a SelectOperator." in Hive queries
+    * [HIVE-17417] - LazySimple Timestamp is very expensive
+    * [HIVE-17419] - ANALYZE TABLE...COMPUTE STATISTICS FOR COLUMNS command shows computed stats for masked tables
+    * [HIVE-17420] - bootstrap - get replid before object dump
+    * [HIVE-17421] - Clear incorrect stats after replication
+    * [HIVE-17429] - Hive JDBC doesn't return rows when querying Impala
+    * [HIVE-17450] - rename TestTxnCommandsBase 
+    * [HIVE-17452] - HPL/SQL function variable block is not initialized
+    * [HIVE-17453] - Missing ASF headers 2 classes
+    * [HIVE-17457] - IOW Acid Insert Overwrite when the transaction fails
+    * [HIVE-17459] - View deletion operation failed to replicate on target cluster
+    * [HIVE-17460] - `insert overwrite` should support table schema evolution (e.g. add columns)
+    * [HIVE-17463] - ORC: include orc-shims in hive-exec.jar
+    * [HIVE-17464] - Fix to be able to disable max shuffle size DHJ config
+    * [HIVE-17465] - Statistics: Drill-down filters don't reduce row-counts progressively
+    * [HIVE-17468] - Shade and package appropriate jackson version for druid storage handler
+    * [HIVE-17471] - Vectorization: Enable hive.vectorized.row.identifier.enabled to true by default
+    * [HIVE-17472] - Drop-partition for multi-level partition fails, if data does not exist.
+    * [HIVE-17475] - Disable mapjoin using hint
+    * [HIVE-17479] - Staging directories do not get cleaned up for update/delete queries
+    * [HIVE-17483] - HS2 kill command to kill queries using query id
+    * [HIVE-17485] - Hive-Druid table on indexing for few segments- DruidRecordWriter.pushSegments throws ArrayIndexOutOfBoundsException
+    * [HIVE-17489] - Separate client-facing and server-side Kerberos principals, to support HA
+    * [HIVE-17496] - Bootstrap repl is not cleaning up staging dirs
+    * [HIVE-17504] - Skip ACID table for replication
+    * [HIVE-17510] - Make comparison of filter predicates in q files deterministic
+    * [HIVE-17512] - Not use doAs if distcp privileged user same as user running hive
+    * [HIVE-17522] - cleanup old 'repl dump' dirs
+    * [HIVE-17523] - Insert into druid table  hangs Hive server2 in an infinite loop
+    * [HIVE-17529] - Bucket Map Join : Sets incorrect edge type causing execution failure
+    * [HIVE-17530] - ClassCastException when converting uniontype
+    * [HIVE-17535] - Select 1 EXCEPT Select 1 fails with NPE
+    * [HIVE-17553] - CBO wrongly type cast decimal literal to int
+    * [HIVE-17554] - Occurr java.lang.ArithmeticException: / by zero at hplsql component
+    * [HIVE-17556] - The test udf_mask_hash.q is failing
+    * [HIVE-17558] - Skip non-native/temporary tables for constraint related scenarios
+    * [HIVE-17560] - HiveMetastore doesn't start in secure cluster if repl change manager is enabled
+    * [HIVE-17563] - CodahaleMetrics.JsonFileReporter is not updating hive.service.metrics.file.location
+    * [HIVE-17568] - HiveJoinPushTransitivePredicatesRule may exchange predicates which are not valid on the other branch
+    * [HIVE-17571] - update sql standard authorization config whitelist to include distcp options for replication
+    * [HIVE-17576] - Improve progress-reporting in TezProcessor
+    * [HIVE-17582] - Followup of HIVE-15708
+    * [HIVE-17584] - fix mapred.job.queue.name in sql standard authorization config whitelist
+    * [HIVE-17585] - Improve thread safety when loading dynamic partitions in parallel
+    * [HIVE-17588] - LlapRowRecordReader doing name-based field lookup for every column of every row
+    * [HIVE-17594] - Unit format error in Copy.java
+    * [HIVE-17595] - Correct DAG for updating the last.repl.id for a database during bootstrap load
+    * [HIVE-17601] - improve error handling in LlapServiceDriver
+    * [HIVE-17602] - Explain plan not working
+    * [HIVE-17610] - LLAP IO: an exception in exception handling can hide the original exception
+    * [HIVE-17613] - remove object pools for short, same-thread allocations
+    * [HIVE-17615] - Task.executeTask has to be thread safe for parallel execution
+    * [HIVE-17619] - Exclude avatica-core.jar dependency from avatica shaded jar
+    * [HIVE-17620] - Use the default MR scratch directory (HDFS) in the only case when hive.blobstore.optimizations.enabled=true AND isFinalJob=true
+    * [HIVE-17621] - Hive-site settings are ignored during HCatInputFormat split-calculation
+    * [HIVE-17623] - Fix Select query Fix Double column serde and some refactoring
+    * [HIVE-17624] - MapredLocakTask running in separate JVM could throw ClassNotFoundException 
+    * [HIVE-17625] - Replication: update hive.repl.partitions.dump.parallelism to 100
+    * [HIVE-17627] - Use druid scan query instead of the select query.
+    * [HIVE-17628] - always use fully qualified path for tables/partitions/etc.
+    * [HIVE-17633] - Make it possible to override the query results directory in TestBeeLineDriver
+    * [HIVE-17635] - Add unit tests to CompactionTxnHandler and use PreparedStatements for queries
+    * [HIVE-17639] - don't reuse planner context when re-parsing the query
+    * [HIVE-17643] - recent WM changes broke reopen due to spurious overloads
+    * [HIVE-17644] - directSQL errors out on key constraints until the DB is initialized
+    * [HIVE-17649] - Export/Import: Move export data write to a task
+    * [HIVE-17653] - Druid storage handler CTAS with boolean type columns fails. 
+    * [HIVE-17659] - get_token thrift call fails for DBTokenStore in remote HMS mode
+    * [HIVE-17664] - Refactor and add new tests
+    * [HIVE-17665] - Update netty-all to latest 4.0.x.Final
+    * [HIVE-17679] - http-generic-click-jacking for WebHcat server
+    * [HIVE-17682] - Vectorization: IF stmt produces wrong results
+    * [HIVE-17690] - Add distcp.options.p* in sql standard authorization config whitelist
+    * [HIVE-17701] - Added restriction to historic queries on web UI
+    * [HIVE-17702] - incorrect isRepeating handling in decimal reader in ORC
+    * [HIVE-17706] - Add a possibility to run the BeeLine tests on the default database
+    * [HIVE-17715] - Exception when pushing postaggregates into Druid
+    * [HIVE-17720] - Bitvectors are not shown in describe statement on beeline
+    * [HIVE-17721] - with Postgres rdbms for metastore and dbnotification enabled, hive DDL SQL query fails 
+    * [HIVE-17723] - Update Accumulo drive q.out files
+    * [HIVE-17725] - Fix misnamed tests which are not run during precommit runs. 
+    * [HIVE-17726] - Using exists may lead to incorrect results
+    * [HIVE-17731] - add a backward compat option for external users to HIVE-11985
+    * [HIVE-17735] - ObjectStore.addNotificationEvent is leaking queries
+    * [HIVE-17746] - Regenerate spark_explainuser_1.q.out
+    * [HIVE-17749] - Multiple class have missed the ASF header
+    * [HIVE-17758] - NOTIFICATION_SEQUENCE_LOCK_RETRY_SLEEP_INTERVAL.defaultLongVal is -1
+    * [HIVE-17761] - Deprecate hive.druid.select.distribute property for Druid
+    * [HIVE-17762] - Exclude older jackson-annotation.jar from druid-handler shaded jar
+    * [HIVE-17764] - alter view fails when hive.metastore.disallow.incompatible.col.type.changes set to true
+    * [HIVE-17765] - expose Hive keywords 
+    * [HIVE-17777] - Add maven coordinates in itests/pom.xml
+    * [HIVE-17781] - Map MR settings to Tez settings via DeprecatedKeys
+    * [HIVE-17782] - Inconsistent cast behavior from string to numeric types with regards to leading/trailing spaces
+    * [HIVE-17785] - Encription tests are not running
+    * [HIVE-17792] - Enable Bucket Map Join when there are extra keys other than bucketed columns
+    * [HIVE-17795] - Add distribution management tag in pom
+    * [HIVE-17798] - When replacing the src table names in BeeLine testing, the table names shouldn't be changed to lower case
+    * [HIVE-17803] - With Pig multi-query, 2 HCatStorers writing to the same table will trample each other's outputs
+    * [HIVE-17804] - Vectorization: Bug erroneously causes match for 1st row in batch (SelectStringColLikeStringScalar)
+    * [HIVE-17806] - Create directory for metrics file if it doesn't exist
+    * [HIVE-17807] - Execute maven commands in batch mode for ptests
+    * [HIVE-17813] - hive.exec.move.files.from.source.dir does not work with partitioned tables
+    * [HIVE-17815] - prevent OOM with Atlas Hive hook 
+    * [HIVE-17817] - Stabilize crossproduct warning message output order
+    * [HIVE-17822] - Provide an option to skip shading of jars
+    * [HIVE-17825] - Socket not closed when trying to read files to copy over in replication from metadata
+    * [HIVE-17826] - Error writing to RandomAccessFile after operation log is closed
+    * [HIVE-17828] - Metastore: mysql upgrade scripts to 3.0.0 is broken
+    * [HIVE-17829] - ArrayIndexOutOfBoundsException - HBASE-backed tables with Avro schema in Hive2
+    * [HIVE-17830] - dbnotification fails to work with rdbms other than postgres
+    * [HIVE-17831] - HiveSemanticAnalyzerHookContext does not update the HiveOperation after sem.analyze() is called
+    * [HIVE-17832] - Allow hive.metastore.disallow.incompatible.col.type.changes to be changed in metastore
+    * [HIVE-17833] - Publish split generation counters
+    * [HIVE-17834] - Fix flaky triggers test
+    * [HIVE-17836] - Persisting nulls in bit vector field fails for postgres backed metastore
+    * [HIVE-17839] - Cannot generate thrift definitions in standalone-metastore
+    * [HIVE-17843] - UINT32 Parquet columns are handled as signed INT32-s, silently reading incorrect data
+    * [HIVE-17845] - insert fails if target table columns are not lowercase
+    * [HIVE-17853] - RetryingMetaStoreClient loses UGI impersonation-context when reconnecting after timeout
+    * [HIVE-17864] - PTestClient cannot start during Precommit tests
+    * [HIVE-17867] - Exception in windowing functions with TIMESTAMP WITH LOCAL TIME ZONE type
+    * [HIVE-17868] - Make queries in spark_local_queries.q have deterministic output
+    * [HIVE-17872] - Ignoring schema autostart doesn't work (HIVE-14152 used the wrong setting)
+    * [HIVE-17873] - External LLAP client: allow same handleID to be used more than once
+    * [HIVE-17882] - Resource plan retrieval looks incorrect
+    * [HIVE-17891] - HIVE-13076 uses create table if not exists for the postgres script
+    * [HIVE-17900] - analyze stats on columns triggered by Compactor generates malformed SQL with > 1 partition column
+    * [HIVE-17908] - LLAP External client not correctly handling killTask for pending requests
+    * [HIVE-17918] - NPE during semijoin reduction optimization when LLAP caching disabled
+    * [HIVE-17936] - Dynamic Semijoin Reduction : markSemiJoinForDPP marks unwanted semijoin branches
+    * [HIVE-17937] - llap_acid_fast test is flaky
+    * [HIVE-17939] - Bucket map join not being selected when bucketed tables is missing bucket files
+    * [HIVE-17942] - HiveAlterHandler not using conf from HMS Handler
+    * [HIVE-17952] - Fix license headers to avoid dangling javadoc warnings
+    * [HIVE-17953] - Metrics should move to destination atomically
+    * [HIVE-17963] - Fix for HIVE-17113 can be improved for non-blobstore filesystems
+    * [HIVE-17966] - org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveArrayInspector - Review
+    * [HIVE-17973] - Fix small bug in multi_insert_union_src.q
+    * [HIVE-17976] - HoS: don't set output collector if there's no data to process
+    * [HIVE-17978] - Shared work optimizer may leave useless operator branches in the plan
+    * [HIVE-17994] - Vectorization: Serialization bottlenecked on irrelevant hashmap lookup
+    * [HIVE-18001] - InvalidObjectException while creating Primary Key constraint on partition key column
+    * [HIVE-18006] - Optimize memory footprint of HLLDenseRegister
+    * [HIVE-18007] - Address maven warnings
+    * [HIVE-18012] - fix ct_noperm_loc test
+    * [HIVE-18016] - org.apache.hadoop.hive.ql.util.ResourceDownloader - Review
+    * [HIVE-18017] - HS2 materialized view registry init loading all tables from metastore
+    * [HIVE-18026] - Hive webhcat principal configuration optimization
+    * [HIVE-18046] - Metastore: default IS_REWRITE_ENABLED=false instead of NULL
+    * [HIVE-18050] - LlapServiceDriver shoud split HIVE_AUX_JARS_PATH by ':' instead of ','
+    * [HIVE-18054] -  Make Lineage work with concurrent queries on a Session
+    * [HIVE-18060] - UpdateInputAccessTimeHook fails for non-current database
+    * [HIVE-18067] - Remove extraneous golden files
+    * [HIVE-18068] - Upgrade to Calcite 1.15
+    * [HIVE-18069] - MetaStoreDirectSql to get tables has misplaced comma
+    * [HIVE-18077] - Vectorization: Add string conversion case for UDFToDouble
+    * [HIVE-18090] - acid heartbeat fails when metastore is connected via hadoop credential
+    * [HIVE-18109] - fix identifier usage in parser
+    * [HIVE-18111] - Fix temp path for Spark DPP sink
+    * [HIVE-18124] -  clean up isAcidTable() API vs isInsertOnlyTable()
+    * [HIVE-18127] - Do not strip '--' comments from shell commands issued from CliDriver
+    * [HIVE-18136] - WorkloadManagerMxBean is missing the Apache license header
+    * [HIVE-18146] - Vectorization: VectorMapJoinOperator Decimal64ColumnVector key/value cast bug
+    * [HIVE-18147] - Tests can fail with java.net.BindException: Address already in use
+    * [HIVE-18148] - NPE in SparkDynamicPartitionPruningResolver
+    * [HIVE-18150] - Upgrade Spark Version to 2.2.0
+    * [HIVE-18151] - LLAP external client: Better error message propagation during submission failures
+    * [HIVE-18157] - Vectorization : Insert in bucketed table is broken with vectorization
+    * [HIVE-18160] - Jar localization during session initialization is slow
+    * [HIVE-18166] - Result of hive.query.string is encoded.
+    * [HIVE-18188] - Fix TestSSL failures in master
+    * [HIVE-18189] - Order by position does not work when cbo is disabled
+    * [HIVE-18191] - Vectorization: Add validation of TableScanOperator (gather statistics) back
+    * [HIVE-18194] - Migrate existing ACID tables to use write id per table rather than global transaction id
+    * [HIVE-18195] - Hive schema broken on postgres
+    * [HIVE-18196] - Druid Mini Cluster to run Qtests integrations tests.
+    * [HIVE-18198] - TablePropertyEnrichmentOptimizer.java is missing the Apache license header
+    * [HIVE-18207] - Fix the test failure for TestCliDriver#vector_complex_join
+    * [HIVE-18208] - SMB Join : Fix the unit tests to run SMB Joins.
+    * [HIVE-18210] - create resource plan allows duplicates
+    * [HIVE-18213] - Tests: YARN Minicluster times out if the disks are >90% full
+    * [HIVE-18220] - Workload Management tables have broken constraints defined on postgres schema
+    * [HIVE-18228] - Azure credential properties should be added to the HiveConf hidden list
+    * [HIVE-18232] - Packaging: add dfs-init script in package target
+    * [HIVE-18240] - support getClientInfo/setClientInfo in JDBC
+    * [HIVE-18241] - Query with LEFT SEMI JOIN producing wrong result
+    * [HIVE-18248] - Clean up parameters
+    * [HIVE-18250] - CBO gets turned off with duplicates in RR error
+    * [HIVE-18254] - Use proper AVG Calcite primitive instead of Other_FUNCTION
+    * [HIVE-18255] - spark-client jar should be prefixed with hive-
+    * [HIVE-18258] - Vectorization: Reduce-Side GROUP BY MERGEPARTIAL with duplicate columns is broken
+    * [HIVE-18263] - Ptest execution are multiple times slower sometimes due to dying executor slaves
+    * [HIVE-18266] - LLAP: /system references wrong file for THP
+    * [HIVE-18269] - LLAP: Fast llap io with slow processing pipeline can lead to OOM
+    * [HIVE-18271] - Druid Insert into fails with exception when committing files
+    * [HIVE-18290] - hbase backed table creation fails where no column comments present
+    * [HIVE-18293] - Hive is failing to compact tables contained within a folder that is not owned by identity running HiveMetaStore
+    * [HIVE-18298] - Fix TestReplicationScenarios.testConstraints
+    * [HIVE-18299] - DbNotificationListener fail on mysql with "select for update"
+    * [HIVE-18306] - Fix spark smb tests
+    * [HIVE-18309] - qtests: smb_mapjoin_19.q breaks bucketsortoptimize_insert_2.q
+    * [HIVE-18310] - Test 'vector_reduce_groupby_duplicate_cols.q' is misspelled in testconfiguration.properties
+    * [HIVE-18311] - Enable smb_mapjoin_8.q for cli driver
+    * [HIVE-18314] - qtests: semijoin_hint.q breaks hybridgrace_hashjoin_2.q	
+    * [HIVE-18316] - HiveEndPoint should only work with full acid tables
+    * [HIVE-18318] - LLAP record reader should check interrupt even when not blocking
+    * [HIVE-18321] - Support REBUILD for MVs backed by custom storage handlers
+    * [HIVE-18326] - LLAP Tez scheduler - only preempt tasks if there's a dependency between them
+    * [HIVE-18330] - Fix TestMsgBusConnection - doesn't test tests the original intention
+    * [HIVE-18331] - Renew the Kerberos ticket used by Druid Query runner
+    * [HIVE-18335] - Vectorization : Check bounds of array before the allocation in VectorMapJoinFastBytesHashTable
+    * [HIV

<TRUNCATED>

[03/50] [abbrv] hive git commit: HIVE-19357: Vectorization: assert_true HiveException erroneously gets suppressed to NULL (Matt McCline, reviewed by Zoltan Haindrich)

Posted by vg...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncDecimalToDouble.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncDecimalToDouble.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncDecimalToDouble.java
index 2d8becf..0921550 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncDecimalToDouble.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncDecimalToDouble.java
@@ -24,6 +24,7 @@ import org.apache.hadoop.hive.ql.exec.vector.DecimalColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.DoubleColumnVector;
 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;
 
 /**
  * This is a superclass for unary decimal functions and expressions returning doubles that
@@ -48,7 +49,7 @@ public abstract class FuncDecimalToDouble extends VectorExpression {
   abstract protected void func(DoubleColumnVector outputColVector, DecimalColumnVector inputColVector, int i);
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     if (childExpressions != null) {
       super.evaluateChildren(batch);

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncDecimalToLong.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncDecimalToLong.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncDecimalToLong.java
index 0ef3da0..6b90ec5 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncDecimalToLong.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncDecimalToLong.java
@@ -24,6 +24,7 @@ import org.apache.hadoop.hive.ql.exec.vector.DecimalColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector.PrimitiveCategory;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils;
@@ -56,7 +57,7 @@ public abstract class FuncDecimalToLong extends VectorExpression {
   abstract protected void func(LongColumnVector outputColVector, DecimalColumnVector inputColVector, int i);
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     if (childExpressions != null) {
       super.evaluateChildren(batch);

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncDecimalToTimestamp.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncDecimalToTimestamp.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncDecimalToTimestamp.java
index 8324506..01d710d 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncDecimalToTimestamp.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncDecimalToTimestamp.java
@@ -24,6 +24,7 @@ import org.apache.hadoop.hive.ql.exec.vector.DecimalColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.TimestampColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 /**
  * This is a superclass for unary decimal functions and expressions returning timestamps that
@@ -49,7 +50,7 @@ public abstract class FuncDecimalToTimestamp extends VectorExpression {
   abstract protected void func(TimestampColumnVector outputColVector, DecimalColumnVector inputColVector, int i);
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     if (childExpressions != null) {
       super.evaluateChildren(batch);

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncDoubleToDecimal.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncDoubleToDecimal.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncDoubleToDecimal.java
index b67632a..7fc8f6a 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncDoubleToDecimal.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncDoubleToDecimal.java
@@ -24,6 +24,7 @@ import org.apache.hadoop.hive.ql.exec.vector.DecimalColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.DoubleColumnVector;
 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;
 
 /**
  * This is a superclass for unary double functions and expressions returning decimals that
@@ -48,7 +49,7 @@ public abstract class FuncDoubleToDecimal extends VectorExpression {
   abstract protected void func(DecimalColumnVector outputColVector, DoubleColumnVector inputColVector, int i);
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     if (childExpressions != null) {
       super.evaluateChildren(batch);

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncLongToDecimal.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncLongToDecimal.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncLongToDecimal.java
index 1b8707e..d3136c3 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncLongToDecimal.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncLongToDecimal.java
@@ -24,6 +24,7 @@ import org.apache.hadoop.hive.ql.exec.vector.DecimalColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 /**
  * This is a superclass for unary long functions and expressions returning decimals that
@@ -48,7 +49,7 @@ public abstract class FuncLongToDecimal extends VectorExpression {
   abstract protected void func(DecimalColumnVector outputColVector, LongColumnVector inputColVector, int i);
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     if (childExpressions != null) {
       super.evaluateChildren(batch);

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncLongToString.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncLongToString.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncLongToString.java
index 733444e..5805afc 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncLongToString.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncLongToString.java
@@ -61,7 +61,7 @@ public abstract class FuncLongToString extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     if (childExpressions != null) {
       super.evaluateChildren(batch);

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncRand.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncRand.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncRand.java
index aebfa25..d6d52ed 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncRand.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncRand.java
@@ -24,6 +24,7 @@ import java.util.Random;
 import org.apache.hadoop.hive.ql.exec.vector.DoubleColumnVector;
 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;
 
 /**
  * Implements vectorized rand(seed) function evaluation.
@@ -46,7 +47,7 @@ public class FuncRand extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     if (childExpressions != null) {
       this.evaluateChildren(batch);

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncRandNoSeed.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncRandNoSeed.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncRandNoSeed.java
index f0d7c60..a432d3d 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncRandNoSeed.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncRandNoSeed.java
@@ -24,6 +24,7 @@ import java.util.Random;
 import org.apache.hadoop.hive.ql.exec.vector.DoubleColumnVector;
 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;
 
 /**
  * Implements vectorized rand() function evaluation.
@@ -46,7 +47,7 @@ public class FuncRandNoSeed extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     if (childExpressions != null) {
       this.evaluateChildren(batch);

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncRoundWithNumDigitsDecimalToDecimal.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncRoundWithNumDigitsDecimalToDecimal.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncRoundWithNumDigitsDecimalToDecimal.java
index e605e88..bad5101 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncRoundWithNumDigitsDecimalToDecimal.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncRoundWithNumDigitsDecimalToDecimal.java
@@ -23,6 +23,7 @@ import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
 import org.apache.hadoop.hive.ql.exec.vector.DecimalColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.DecimalUtil;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
 
 import java.util.Arrays;
@@ -48,7 +49,7 @@ public class FuncRoundWithNumDigitsDecimalToDecimal extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     if (childExpressions != null) {
       this.evaluateChildren(batch);

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncStringToLong.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncStringToLong.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncStringToLong.java
index 1b5c07a..fffb36d 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncStringToLong.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncStringToLong.java
@@ -24,6 +24,7 @@ import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
 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;
 
 /**
  * Superclass to support vectorized functions that take a long
@@ -46,7 +47,7 @@ public abstract class FuncStringToLong extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     if (childExpressions != null) {
       super.evaluateChildren(batch);

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncTimestampToDecimal.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncTimestampToDecimal.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncTimestampToDecimal.java
index 2213b83..53b3d83 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncTimestampToDecimal.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncTimestampToDecimal.java
@@ -24,6 +24,7 @@ import org.apache.hadoop.hive.ql.exec.vector.DecimalColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.TimestampColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 /**
  * This is a superclass for unary timestamp functions and expressions returning decimals that
@@ -49,7 +50,7 @@ public abstract class FuncTimestampToDecimal extends VectorExpression {
   abstract protected void func(DecimalColumnVector outputColVector, TimestampColumnVector inputColVector, int i);
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     if (childExpressions != null) {
       super.evaluateChildren(batch);

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncTimestampToLong.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncTimestampToLong.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncTimestampToLong.java
index 060070e..c02ad5d 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncTimestampToLong.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncTimestampToLong.java
@@ -25,6 +25,7 @@ import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.TimestampColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 /**
  * This is a superclass for unary timestamp functions and expressions returning long that
@@ -50,7 +51,7 @@ public abstract class FuncTimestampToLong extends VectorExpression {
   abstract protected void func(LongColumnVector outputColVector, TimestampColumnVector inputColVector, int i);
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     if (childExpressions != null) {
       super.evaluateChildren(batch);

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IdentityExpression.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IdentityExpression.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IdentityExpression.java
index 86d3dbf..8f24027 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IdentityExpression.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IdentityExpression.java
@@ -19,6 +19,7 @@ package org.apache.hadoop.hive.ql.exec.vector.expressions;
 
 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;
 
 /**
  * An expression representing a column, only children are evaluated.
@@ -35,7 +36,7 @@ public class IdentityExpression extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
     if (childExpressions != null) {
       this.evaluateChildren(batch);
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprColumnCondExpr.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprColumnCondExpr.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprColumnCondExpr.java
index 94e5190..0a1fa17 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprColumnCondExpr.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprColumnCondExpr.java
@@ -19,6 +19,7 @@ package org.apache.hadoop.hive.ql.exec.vector.expressions;
 
 import org.apache.hadoop.hive.ql.exec.vector.ColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 /**
  * Do regular execution of the THEN vector expression (a column or scalar) and conditional execution
@@ -46,7 +47,7 @@ public class IfExprColumnCondExpr extends IfExprCondExprBase {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     int n = batch.size;
     if (n <= 0) {

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprColumnNull.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprColumnNull.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprColumnNull.java
index bf2c9a4..7d56ed3 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprColumnNull.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprColumnNull.java
@@ -21,6 +21,7 @@ import org.apache.hadoop.hive.ql.exec.vector.ColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
 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;
 
 public class IfExprColumnNull extends VectorExpression {
 
@@ -44,7 +45,7 @@ public class IfExprColumnNull extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     if (childExpressions != null) {
       super.evaluateChildren(batch);

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprCondExprBase.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprCondExprBase.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprCondExprBase.java
index abc1343..04b54a2 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprCondExprBase.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprCondExprBase.java
@@ -20,6 +20,7 @@ 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.VectorExpressionDescriptor;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 /**
  * Base class that supports conditional execution of the THEN/ELSE vector expressions of
@@ -54,7 +55,7 @@ public abstract class IfExprCondExprBase extends VectorExpression {
   }
 
   public void conditionalEvaluate(VectorizedRowBatch batch, VectorExpression condVecExpr,
-      int[] condSelected, int condSize) {
+      int[] condSelected, int condSize) throws HiveException {
 
     int saveSize = batch.size;
     boolean saveSelectedInUse = batch.selectedInUse;
@@ -72,7 +73,7 @@ public abstract class IfExprCondExprBase extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     // NOTE: We do conditional vector expression so we do not call super.evaluateChildren(batch).
 

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprCondExprColumn.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprCondExprColumn.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprCondExprColumn.java
index cc465c1..f2f4d3e 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprCondExprColumn.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprCondExprColumn.java
@@ -19,6 +19,7 @@ package org.apache.hadoop.hive.ql.exec.vector.expressions;
 
 import org.apache.hadoop.hive.ql.exec.vector.ColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 /**
  * Do conditional execution of the THEN/ vector expression and regular execution of the ELSE
@@ -46,7 +47,7 @@ public class IfExprCondExprColumn extends IfExprCondExprBase {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     int n = batch.size;
     if (n <= 0) {

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprCondExprCondExpr.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprCondExprCondExpr.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprCondExprCondExpr.java
index 7874d5c..59db6b7 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprCondExprCondExpr.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprCondExprCondExpr.java
@@ -19,6 +19,7 @@ package org.apache.hadoop.hive.ql.exec.vector.expressions;
 
 import org.apache.hadoop.hive.ql.exec.vector.ColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 /**
  * Do conditional execution of the THEN/ELSE vector expressions of a SQL IF statement.
@@ -45,7 +46,7 @@ public class IfExprCondExprCondExpr extends IfExprCondExprBase {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     int n = batch.size;
     if (n <= 0) {

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprCondExprNull.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprCondExprNull.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprCondExprNull.java
index b2bf0e4..40dbb38 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprCondExprNull.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprCondExprNull.java
@@ -19,6 +19,7 @@ package org.apache.hadoop.hive.ql.exec.vector.expressions;
 
 import org.apache.hadoop.hive.ql.exec.vector.ColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 /**
  * Do conditional execution of the THEN vector expression with NULL ELSE of a SQL IF statement.
@@ -41,7 +42,7 @@ public class IfExprCondExprNull extends IfExprCondExprBase {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     int n = batch.size;
     if (n <= 0) {

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprDoubleColumnDoubleColumn.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprDoubleColumnDoubleColumn.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprDoubleColumnDoubleColumn.java
index 4296692..3772270 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprDoubleColumnDoubleColumn.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprDoubleColumnDoubleColumn.java
@@ -21,6 +21,7 @@ import org.apache.hadoop.hive.ql.exec.vector.DoubleColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.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.
@@ -53,7 +54,7 @@ public class IfExprDoubleColumnDoubleColumn extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     if (childExpressions != null) {
       super.evaluateChildren(batch);

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/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
index 099a319..13e5fff 100644
--- 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
@@ -21,6 +21,7 @@ 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.
@@ -53,7 +54,7 @@ public class IfExprIntervalDayTimeColumnColumn extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     if (childExpressions != null) {
       super.evaluateChildren(batch);

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/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
index 905ffba..d5fb6a0 100644
--- 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
@@ -25,6 +25,7 @@ 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.
@@ -58,7 +59,7 @@ public class IfExprIntervalDayTimeColumnScalar extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     if (childExpressions != null) {
       super.evaluateChildren(batch);

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/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
index e99754b..53466e5 100644
--- 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
@@ -25,6 +25,7 @@ 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.
@@ -58,7 +59,7 @@ public class IfExprIntervalDayTimeScalarColumn extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     if (childExpressions != null) {
       super.evaluateChildren(batch);

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/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
index 5875d48..177f358 100644
--- 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
@@ -25,6 +25,7 @@ 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.
@@ -58,7 +59,7 @@ public class IfExprIntervalDayTimeScalarScalar extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     if (childExpressions != null) {
       super.evaluateChildren(batch);

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/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 d8ec895..75de7a0 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
@@ -20,6 +20,7 @@ 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.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.
@@ -52,7 +53,7 @@ public class IfExprLongColumnLongColumn extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     if (childExpressions != null) {
       super.evaluateChildren(batch);

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprNullColumn.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprNullColumn.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprNullColumn.java
index 4afdce4..a4c44c3 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprNullColumn.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprNullColumn.java
@@ -21,6 +21,7 @@ import org.apache.hadoop.hive.ql.exec.vector.ColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
 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;
 
 public class IfExprNullColumn extends VectorExpression {
 
@@ -44,7 +45,7 @@ public class IfExprNullColumn extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     if (childExpressions != null) {
       super.evaluateChildren(batch);

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprNullCondExpr.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprNullCondExpr.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprNullCondExpr.java
index 2ca3388..aa971c3 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprNullCondExpr.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprNullCondExpr.java
@@ -19,6 +19,7 @@ package org.apache.hadoop.hive.ql.exec.vector.expressions;
 
 import org.apache.hadoop.hive.ql.exec.vector.ColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 /**
  * Do conditional execution of a NULL THEN and a ELSE vector expression of a SQL IF statement.
@@ -41,7 +42,7 @@ public class IfExprNullCondExpr extends IfExprCondExprBase {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     int n = batch.size;
     if (n <= 0) {

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprNullNull.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprNullNull.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprNullNull.java
index 5a68cec..c5a4bd3 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprNullNull.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprNullNull.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.hive.ql.exec.vector.expressions;
 import org.apache.hadoop.hive.ql.exec.vector.ColumnVector;
 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;
 
 public class IfExprNullNull extends VectorExpression {
 
@@ -34,7 +35,7 @@ public class IfExprNullNull extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     if (childExpressions != null) {
       super.evaluateChildren(batch);

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprStringGroupColumnStringGroupColumn.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprStringGroupColumnStringGroupColumn.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprStringGroupColumnStringGroupColumn.java
index bb57e4e..fc244ec 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprStringGroupColumnStringGroupColumn.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprStringGroupColumnStringGroupColumn.java
@@ -23,6 +23,7 @@ import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
 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.
@@ -55,7 +56,7 @@ public class IfExprStringGroupColumnStringGroupColumn extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     if (childExpressions != null) {
       super.evaluateChildren(batch);

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprStringGroupColumnStringScalar.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprStringGroupColumnStringScalar.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprStringGroupColumnStringScalar.java
index 998448a..e6b30d9 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprStringGroupColumnStringScalar.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprStringGroupColumnStringScalar.java
@@ -25,6 +25,7 @@ import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
 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 expressions.
@@ -58,7 +59,7 @@ public class IfExprStringGroupColumnStringScalar extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     if (childExpressions != null) {
       super.evaluateChildren(batch);

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprStringScalarStringGroupColumn.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprStringScalarStringGroupColumn.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprStringScalarStringGroupColumn.java
index c597a34..52ed087 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprStringScalarStringGroupColumn.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprStringScalarStringGroupColumn.java
@@ -25,6 +25,7 @@ import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
 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.
@@ -59,7 +60,7 @@ public class IfExprStringScalarStringGroupColumn extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     if (childExpressions != null) {
       super.evaluateChildren(batch);

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprStringScalarStringScalar.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprStringScalarStringScalar.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprStringScalarStringScalar.java
index 9c0e7be..9992743 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprStringScalarStringScalar.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprStringScalarStringScalar.java
@@ -26,6 +26,7 @@ import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
 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.
@@ -59,7 +60,7 @@ public class IfExprStringScalarStringScalar extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     if (childExpressions != null) {
       super.evaluateChildren(batch);

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/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
index ed21ce9..fc7a859 100644
--- 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
@@ -20,6 +20,7 @@ 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.
@@ -52,7 +53,7 @@ public abstract class IfExprTimestampColumnColumnBase extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     if (childExpressions != null) {
       super.evaluateChildren(batch);

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/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
index c0cb2c1..64add36 100644
--- 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
@@ -25,6 +25,7 @@ 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.
@@ -58,7 +59,7 @@ public abstract class IfExprTimestampColumnScalarBase extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     if (childExpressions != null) {
       super.evaluateChildren(batch);

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/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
index 0798f1f..73044ad 100644
--- 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
@@ -26,6 +26,7 @@ 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.
@@ -59,7 +60,7 @@ public abstract class IfExprTimestampScalarColumnBase extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     if (childExpressions != null) {
       super.evaluateChildren(batch);

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/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
index 0059c58..df1418e 100644
--- 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
@@ -22,6 +22,7 @@ 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;
@@ -58,7 +59,7 @@ public abstract class IfExprTimestampScalarScalarBase extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     if (childExpressions != null) {
       super.evaluateChildren(batch);

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IsNotNull.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IsNotNull.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IsNotNull.java
index a5cddc6..6f41b42 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IsNotNull.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IsNotNull.java
@@ -23,6 +23,7 @@ import org.apache.hadoop.hive.ql.exec.vector.ColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
 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;
 
 /**
  * This expression evaluates to true if the given input columns is not null.
@@ -46,7 +47,7 @@ public class IsNotNull extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     if (childExpressions != null) {
       super.evaluateChildren(batch);

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IsNull.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IsNull.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IsNull.java
index 17d567f..f700054 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IsNull.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IsNull.java
@@ -23,6 +23,7 @@ import org.apache.hadoop.hive.ql.exec.vector.ColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
 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;
 
 /**
  * This expression evaluates to true if the given input columns is null.
@@ -46,7 +47,7 @@ public class IsNull extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     if (childExpressions != null) {
       super.evaluateChildren(batch);

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/ListIndexColColumn.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/ListIndexColColumn.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/ListIndexColColumn.java
index 9d22a3c..55417cf 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/ListIndexColColumn.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/ListIndexColColumn.java
@@ -23,6 +23,7 @@ import org.apache.hadoop.hive.ql.exec.vector.ListColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
 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;
 
 /**
  * Vectorized instruction to get an element from a list with the index from another column and put
@@ -45,7 +46,7 @@ public class ListIndexColColumn extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
     if (childExpressions != null) {
       super.evaluateChildren(batch);
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/ListIndexColScalar.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/ListIndexColScalar.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/ListIndexColScalar.java
index 948652a..808e9fb 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/ListIndexColScalar.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/ListIndexColScalar.java
@@ -22,6 +22,7 @@ import org.apache.hadoop.hive.ql.exec.vector.ColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.ListColumnVector;
 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;
 
 /**
  * Vectorized instruction to get an element from a list with a scalar index and put
@@ -44,7 +45,7 @@ public class ListIndexColScalar extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
     if (childExpressions != null) {
       super.evaluateChildren(batch);
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColDivideLongColumn.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColDivideLongColumn.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColDivideLongColumn.java
index 42483c0..1ed724c 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColDivideLongColumn.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColDivideLongColumn.java
@@ -22,6 +22,7 @@ import org.apache.hadoop.hive.ql.exec.vector.DoubleColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 /**
  * This operation is handled as a special case because Hive
@@ -49,7 +50,7 @@ public class LongColDivideLongColumn extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     if (childExpressions != null) {
       super.evaluateChildren(batch);

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColDivideLongScalar.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColDivideLongScalar.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColDivideLongScalar.java
index 67d1e76..6445446 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColDivideLongScalar.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColDivideLongScalar.java
@@ -24,6 +24,7 @@ import org.apache.hadoop.hive.ql.exec.vector.DoubleColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 /**
  * This operation is handled as a special case because Hive
@@ -51,7 +52,7 @@ public class LongColDivideLongScalar extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     if (childExpressions != null) {
       super.evaluateChildren(batch);

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColEqualLongColumn.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColEqualLongColumn.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColEqualLongColumn.java
index 608c32a..308ddf9 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColEqualLongColumn.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColEqualLongColumn.java
@@ -21,6 +21,7 @@ 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.VectorExpressionDescriptor;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 public class LongColEqualLongColumn extends VectorExpression {
 
@@ -44,7 +45,7 @@ public class LongColEqualLongColumn extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     if (childExpressions != null) {
       super.evaluateChildren(batch);

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColEqualLongScalar.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColEqualLongScalar.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColEqualLongScalar.java
index 1a82e8e..797dd39 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColEqualLongScalar.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColEqualLongScalar.java
@@ -23,6 +23,7 @@ import java.util.Arrays;
 import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 public class LongColEqualLongScalar extends VectorExpression {
   private static final long serialVersionUID = 1L;
@@ -45,7 +46,7 @@ public class LongColEqualLongScalar extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     // return immediately if batch is empty
     final int n = batch.size;

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColGreaterEqualLongColumn.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColGreaterEqualLongColumn.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColGreaterEqualLongColumn.java
index eb040ca..ad9c7a3 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColGreaterEqualLongColumn.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColGreaterEqualLongColumn.java
@@ -21,6 +21,7 @@ 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.VectorExpressionDescriptor;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 public class LongColGreaterEqualLongColumn extends VectorExpression {
 
@@ -44,7 +45,7 @@ public class LongColGreaterEqualLongColumn extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     if (childExpressions != null) {
       super.evaluateChildren(batch);

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColGreaterEqualLongScalar.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColGreaterEqualLongScalar.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColGreaterEqualLongScalar.java
index 3f0ece7..497826a 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColGreaterEqualLongScalar.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColGreaterEqualLongScalar.java
@@ -23,6 +23,7 @@ import java.util.Arrays;
 import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 public class LongColGreaterEqualLongScalar extends VectorExpression {
 
@@ -46,7 +47,7 @@ public class LongColGreaterEqualLongScalar extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     if (childExpressions != null) {
       super.evaluateChildren(batch);

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColGreaterLongColumn.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColGreaterLongColumn.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColGreaterLongColumn.java
index 9ab9e1e..3e1a13e 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColGreaterLongColumn.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColGreaterLongColumn.java
@@ -21,6 +21,7 @@ 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.VectorExpressionDescriptor;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 public class LongColGreaterLongColumn extends VectorExpression {
 
@@ -44,7 +45,7 @@ public class LongColGreaterLongColumn extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     if (childExpressions != null) {
       super.evaluateChildren(batch);

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColGreaterLongScalar.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColGreaterLongScalar.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColGreaterLongScalar.java
index 4d34707..a4deedf 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColGreaterLongScalar.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColGreaterLongScalar.java
@@ -23,6 +23,7 @@ import java.util.Arrays;
 import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 public class LongColGreaterLongScalar extends VectorExpression {
   private static final long serialVersionUID = 1L;
@@ -45,7 +46,7 @@ public class LongColGreaterLongScalar extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     if (childExpressions != null) {
       super.evaluateChildren(batch);

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColLessEqualLongColumn.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColLessEqualLongColumn.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColLessEqualLongColumn.java
index 004bf4f..3212269 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColLessEqualLongColumn.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColLessEqualLongColumn.java
@@ -21,6 +21,7 @@ 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.VectorExpressionDescriptor;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 public class LongColLessEqualLongColumn extends VectorExpression {
 
@@ -44,7 +45,7 @@ public class LongColLessEqualLongColumn extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     if (childExpressions != null) {
       super.evaluateChildren(batch);

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColLessEqualLongScalar.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColLessEqualLongScalar.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColLessEqualLongScalar.java
index b50bdd3..c2f0de1 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColLessEqualLongScalar.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColLessEqualLongScalar.java
@@ -23,6 +23,7 @@ import java.util.Arrays;
 import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 public class LongColLessEqualLongScalar extends VectorExpression {
 
@@ -46,7 +47,7 @@ public class LongColLessEqualLongScalar extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     if (childExpressions != null) {
       super.evaluateChildren(batch);

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColLessLongColumn.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColLessLongColumn.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColLessLongColumn.java
index 3a3425b..7c2ee20 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColLessLongColumn.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColLessLongColumn.java
@@ -21,6 +21,7 @@ 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.VectorExpressionDescriptor;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 public class LongColLessLongColumn extends VectorExpression {
 
@@ -44,7 +45,7 @@ public class LongColLessLongColumn extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     if (childExpressions != null) {
       super.evaluateChildren(batch);

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColLessLongScalar.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColLessLongScalar.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColLessLongScalar.java
index f32de24..c7efe84 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColLessLongScalar.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColLessLongScalar.java
@@ -23,6 +23,7 @@ import java.util.Arrays;
 import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 public class LongColLessLongScalar extends VectorExpression {
 
@@ -46,7 +47,7 @@ public class LongColLessLongScalar extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     if (childExpressions != null) {
       super.evaluateChildren(batch);

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColModuloLongColumn.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColModuloLongColumn.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColModuloLongColumn.java
index cfd61a9..60faebb 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColModuloLongColumn.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColModuloLongColumn.java
@@ -21,8 +21,7 @@ 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.expressions.NullUtil;
 import org.apache.hadoop.hive.ql.exec.vector.*;
-import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
-import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 /**
  * This operation is handled as a special case because Hive
@@ -51,7 +50,7 @@ public class LongColModuloLongColumn extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     if (childExpressions != null) {
       super.evaluateChildren(batch);

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColModuloLongColumnChecked.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColModuloLongColumnChecked.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColModuloLongColumnChecked.java
index f367139..24a860a 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColModuloLongColumnChecked.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColModuloLongColumnChecked.java
@@ -19,6 +19,7 @@ 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.VectorizedRowBatch;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 /**
  * This vector expression implements a Checked variant of LongColModuloLongColumn
@@ -36,7 +37,7 @@ public class LongColModuloLongColumnChecked extends LongColModuloLongColumn {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
     super.evaluate(batch);
     //checked for overflow based on the outputTypeInfo
     OverflowUtils

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColNotEqualLongColumn.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColNotEqualLongColumn.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColNotEqualLongColumn.java
index 833b8fa..213b876 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColNotEqualLongColumn.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColNotEqualLongColumn.java
@@ -21,6 +21,7 @@ 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.VectorExpressionDescriptor;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 public class LongColNotEqualLongColumn extends VectorExpression {
 
@@ -44,7 +45,7 @@ public class LongColNotEqualLongColumn extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     if (childExpressions != null) {
       super.evaluateChildren(batch);

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColNotEqualLongScalar.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColNotEqualLongScalar.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColNotEqualLongScalar.java
index b4c68fa..c2b52b8 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColNotEqualLongScalar.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColNotEqualLongScalar.java
@@ -23,6 +23,7 @@ import java.util.Arrays;
 import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 public class LongColNotEqualLongScalar extends VectorExpression {
 
@@ -46,7 +47,7 @@ public class LongColNotEqualLongScalar extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     if (childExpressions != null) {
       super.evaluateChildren(batch);

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColumnInList.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColumnInList.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColumnInList.java
index 181aeae..8469882 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColumnInList.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColumnInList.java
@@ -24,6 +24,7 @@ import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
 import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 /**
  * Output a boolean value indicating if a column is IN a list of constants.
@@ -50,7 +51,7 @@ public class LongColumnInList extends VectorExpression implements ILongInExpr {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     if (childExpressions != null) {
       super.evaluateChildren(batch);

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongScalarDivideLongColumn.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongScalarDivideLongColumn.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongScalarDivideLongColumn.java
index df78433..31f4a1c 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongScalarDivideLongColumn.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongScalarDivideLongColumn.java
@@ -24,6 +24,7 @@ import org.apache.hadoop.hive.ql.exec.vector.DoubleColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 /**
  * This operation is handled as a special case because Hive
@@ -51,7 +52,7 @@ public class LongScalarDivideLongColumn extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     if (childExpressions != null) {
       super.evaluateChildren(batch);

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongScalarEqualLongColumn.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongScalarEqualLongColumn.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongScalarEqualLongColumn.java
index 2ca74c3..7b28f71 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongScalarEqualLongColumn.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongScalarEqualLongColumn.java
@@ -23,6 +23,7 @@ import java.util.Arrays;
 import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 public class LongScalarEqualLongColumn extends VectorExpression {
 
@@ -46,7 +47,7 @@ public class LongScalarEqualLongColumn extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     if (childExpressions != null) {
       super.evaluateChildren(batch);

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongScalarGreaterEqualLongColumn.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongScalarGreaterEqualLongColumn.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongScalarGreaterEqualLongColumn.java
index ac245f6..10fd423 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongScalarGreaterEqualLongColumn.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongScalarGreaterEqualLongColumn.java
@@ -23,6 +23,7 @@ import java.util.Arrays;
 import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 public class LongScalarGreaterEqualLongColumn extends VectorExpression {
   private static final long serialVersionUID = 1L;
@@ -45,7 +46,7 @@ public class LongScalarGreaterEqualLongColumn extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     if (childExpressions != null) {
       super.evaluateChildren(batch);

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongScalarGreaterLongColumn.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongScalarGreaterLongColumn.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongScalarGreaterLongColumn.java
index f8cd8a9..ad9ccf5 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongScalarGreaterLongColumn.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongScalarGreaterLongColumn.java
@@ -23,6 +23,7 @@ import java.util.Arrays;
 import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 public class LongScalarGreaterLongColumn extends VectorExpression {
 
@@ -46,7 +47,7 @@ public class LongScalarGreaterLongColumn extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     if (childExpressions != null) {
       super.evaluateChildren(batch);

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongScalarLessEqualLongColumn.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongScalarLessEqualLongColumn.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongScalarLessEqualLongColumn.java
index 7b3d4b3..abe5d08 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongScalarLessEqualLongColumn.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongScalarLessEqualLongColumn.java
@@ -23,6 +23,7 @@ import java.util.Arrays;
 import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 public class LongScalarLessEqualLongColumn extends VectorExpression {
 
@@ -46,7 +47,7 @@ public class LongScalarLessEqualLongColumn extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     if (childExpressions != null) {
       super.evaluateChildren(batch);

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongScalarLessLongColumn.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongScalarLessLongColumn.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongScalarLessLongColumn.java
index 948f812..87ab939 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongScalarLessLongColumn.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongScalarLessLongColumn.java
@@ -23,6 +23,7 @@ import java.util.Arrays;
 import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 public class LongScalarLessLongColumn extends VectorExpression {
 
@@ -46,7 +47,7 @@ public class LongScalarLessLongColumn extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     if (childExpressions != null) {
       super.evaluateChildren(batch);

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongScalarNotEqualLongColumn.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongScalarNotEqualLongColumn.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongScalarNotEqualLongColumn.java
index 1191b31..d936dee 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongScalarNotEqualLongColumn.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongScalarNotEqualLongColumn.java
@@ -23,6 +23,7 @@ import java.util.Arrays;
 import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 public class LongScalarNotEqualLongColumn extends VectorExpression {
 
@@ -46,7 +47,7 @@ public class LongScalarNotEqualLongColumn extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     if (childExpressions != null) {
       super.evaluateChildren(batch);

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongToStringUnaryUDF.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongToStringUnaryUDF.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongToStringUnaryUDF.java
index 0976f20..de0400e 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongToStringUnaryUDF.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongToStringUnaryUDF.java
@@ -24,6 +24,7 @@ import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
 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;
 
 /**
  * This is a superclass for unary long functions returning strings that operate directly on the
@@ -49,7 +50,7 @@ abstract public class LongToStringUnaryUDF extends VectorExpression {
   abstract protected void func(BytesColumnVector outputColVector, long[] vector, int i);
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     if (childExpressions != null) {
       super.evaluateChildren(batch);


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

Posted by vg...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/e2d4f347/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorIfStatement.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorIfStatement.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorIfStatement.java
new file mode 100644
index 0000000..c52ca19
--- /dev/null
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorIfStatement.java
@@ -0,0 +1,444 @@
+/*
+ * 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.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Random;
+
+import org.apache.hadoop.hive.common.type.DataTypePhysicalVariation;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.ql.exec.ExprNodeEvaluator;
+import org.apache.hadoop.hive.ql.exec.ExprNodeEvaluatorFactory;
+import org.apache.hadoop.hive.ql.exec.vector.VectorExtractRow;
+import org.apache.hadoop.hive.ql.exec.vector.VectorRandomBatchSource;
+import org.apache.hadoop.hive.ql.exec.vector.VectorRandomRowSource;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizationContext;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatchCtx;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
+import org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc;
+import org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc;
+import org.apache.hadoop.hive.ql.plan.ExprNodeDesc;
+import org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDF;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDFIf;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDFWhen;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.typeinfo.DecimalTypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils;
+import org.apache.hadoop.io.LongWritable;
+
+import junit.framework.Assert;
+
+import org.junit.Test;
+
+public class TestVectorIfStatement {
+
+  @Test
+  public void testBoolean() throws Exception {
+    Random random = new Random(12882);
+
+    doIfTests(random, "boolean");
+  }
+
+  @Test
+  public void testInt() throws Exception {
+    Random random = new Random(12882);
+
+    doIfTests(random, "int");
+  }
+
+  @Test
+  public void testBigInt() throws Exception {
+    Random random = new Random(12882);
+
+    doIfTests(random, "bigint");
+  }
+
+  @Test
+  public void testString() throws Exception {
+    Random random = new Random(12882);
+
+    doIfTests(random, "string");
+  }
+
+  @Test
+  public void testTimestamp() throws Exception {
+    Random random = new Random(12882);
+
+    doIfTests(random, "timestamp");
+  }
+
+  @Test
+  public void testDate() throws Exception {
+    Random random = new Random(12882);
+
+    doIfTests(random, "date");
+  }
+
+  @Test
+  public void testIntervalDayTime() throws Exception {
+    Random random = new Random(12882);
+
+    doIfTests(random, "interval_day_time");
+  }
+
+  @Test
+  public void testIntervalYearMonth() throws Exception {
+    Random random = new Random(12882);
+
+    doIfTests(random, "interval_year_month");
+  }
+
+  @Test
+  public void testDouble() throws Exception {
+    Random random = new Random(12882);
+
+    doIfTests(random, "double");
+  }
+
+  @Test
+  public void testChar() throws Exception {
+    Random random = new Random(12882);
+
+    doIfTests(random, "char(10)");
+  }
+
+  @Test
+  public void testVarchar() throws Exception {
+    Random random = new Random(12882);
+
+    doIfTests(random, "varchar(15)");
+  }
+
+  @Test
+  public void testBinary() throws Exception {
+    Random random = new Random(12882);
+
+    doIfTests(random, "binary");
+  }
+
+  @Test
+  public void testDecimalLarge() throws Exception {
+    Random random = new Random(9300);
+
+    doIfTests(random, "decimal(20,8)");
+  }
+
+  @Test
+  public void testDecimalSmall() throws Exception {
+    Random random = new Random(12882);
+
+    doIfTests(random, "decimal(10,4)");
+  }
+
+  public enum IfStmtTestMode {
+    ROW_MODE,
+    ADAPTOR_WHEN,
+    VECTOR_EXPRESSION;
+
+    static final int count = values().length;
+  }
+
+  public enum ColumnScalarMode {
+    COLUMN_COLUMN,
+    COLUMN_SCALAR,
+    SCALAR_COLUMN,
+    SCALAR_SCALAR;
+
+    static final int count = values().length;
+  }
+
+  private void doIfTests(Random random, String typeName)
+      throws Exception {
+    doIfTests(random, typeName, DataTypePhysicalVariation.NONE);
+  }
+
+  private void doIfTests(Random random, String typeName,
+      DataTypePhysicalVariation dataTypePhysicalVariation)
+          throws Exception {
+    for (ColumnScalarMode columnScalarMode : ColumnScalarMode.values()) {
+      doIfTestsWithDiffColumnScalar(
+          random, typeName, columnScalarMode, dataTypePhysicalVariation);
+    }
+  }
+
+  private void doIfTestsWithDiffColumnScalar(Random random, String typeName,
+      ColumnScalarMode columnScalarMode, DataTypePhysicalVariation dataTypePhysicalVariation)
+          throws Exception {
+
+    TypeInfo typeInfo = TypeInfoUtils.getTypeInfoFromTypeString(typeName);
+
+    boolean isDecimal64 = (dataTypePhysicalVariation == DataTypePhysicalVariation.DECIMAL_64);
+    final int decimal64Scale =
+        (isDecimal64 ? ((DecimalTypeInfo) typeInfo).getScale() : 0);
+
+    List<String> explicitTypeNameList = new ArrayList<String>();
+    List<DataTypePhysicalVariation> explicitDataTypePhysicalVariationList = new ArrayList<DataTypePhysicalVariation>();
+    explicitTypeNameList.add("boolean");
+    explicitDataTypePhysicalVariationList.add(DataTypePhysicalVariation.NONE);
+    if (columnScalarMode != ColumnScalarMode.SCALAR_SCALAR) {
+      explicitTypeNameList.add(typeName);
+      explicitDataTypePhysicalVariationList.add(dataTypePhysicalVariation);
+      if (columnScalarMode == ColumnScalarMode.COLUMN_COLUMN) {
+        explicitTypeNameList.add(typeName);
+        explicitDataTypePhysicalVariationList.add(dataTypePhysicalVariation);
+      }
+    }
+
+    VectorRandomRowSource rowSource = new VectorRandomRowSource();
+
+    rowSource.initExplicitSchema(
+        random, explicitTypeNameList, /* maxComplexDepth */ 0, /* allowNull */ true,
+        explicitDataTypePhysicalVariationList);
+
+    List<String> columns = new ArrayList<String>();
+    columns.add("col0");    // The boolean predicate.
+
+    ExprNodeColumnDesc col1Expr = new  ExprNodeColumnDesc(Boolean.class, "col0", "table", false);
+    int columnNum = 1;
+    ExprNodeDesc col2Expr;
+    if (columnScalarMode == ColumnScalarMode.COLUMN_COLUMN ||
+        columnScalarMode == ColumnScalarMode.COLUMN_SCALAR) {
+      String columnName = "col" + (columnNum++);
+      col2Expr = new ExprNodeColumnDesc(typeInfo, columnName, "table", false);
+      columns.add(columnName);
+    } else {
+      Object scalar1Object =
+          VectorRandomRowSource.randomPrimitiveObject(
+              random, (PrimitiveTypeInfo) typeInfo);
+      col2Expr = new ExprNodeConstantDesc(typeInfo, scalar1Object);
+    }
+    ExprNodeDesc col3Expr;
+    if (columnScalarMode == ColumnScalarMode.COLUMN_COLUMN ||
+        columnScalarMode == ColumnScalarMode.SCALAR_COLUMN) {
+      String columnName = "col" + (columnNum++);
+      col3Expr = new ExprNodeColumnDesc(typeInfo, columnName, "table", false);
+      columns.add(columnName);
+    } else {
+      Object scalar2Object =
+          VectorRandomRowSource.randomPrimitiveObject(
+              random, (PrimitiveTypeInfo) typeInfo);
+      col3Expr = new ExprNodeConstantDesc(typeInfo, scalar2Object);
+    }
+
+    List<ExprNodeDesc> children = new ArrayList<ExprNodeDesc>();
+    children.add(col1Expr);
+    children.add(col2Expr);
+    children.add(col3Expr);
+
+    //----------------------------------------------------------------------------------------------
+
+    String[] columnNames = columns.toArray(new String[0]);
+
+    String[] outputScratchTypeNames = new String[] { typeName };
+    DataTypePhysicalVariation[] outputDataTypePhysicalVariations =
+        new DataTypePhysicalVariation[] { dataTypePhysicalVariation };
+
+    VectorizedRowBatchCtx batchContext =
+        new VectorizedRowBatchCtx(
+            columnNames,
+            rowSource.typeInfos(),
+            rowSource.dataTypePhysicalVariations(),
+            /* dataColumnNums */ null,
+            /* partitionColumnCount */ 0,
+            /* virtualColumnCount */ 0,
+            /* neededVirtualColumns */ null,
+            outputScratchTypeNames,
+            outputDataTypePhysicalVariations);
+
+    Object[][] randomRows = rowSource.randomRows(100000);
+
+    VectorRandomBatchSource batchSource =
+        VectorRandomBatchSource.createInterestingBatches(
+            random,
+            rowSource,
+            randomRows,
+            null);
+
+    final int rowCount = randomRows.length;
+    Object[][] resultObjectsArray = new Object[IfStmtTestMode.count][];
+    for (int i = 0; i < IfStmtTestMode.count; i++) {
+
+      Object[] resultObjects = new Object[rowCount];
+      resultObjectsArray[i] = resultObjects;
+
+      IfStmtTestMode ifStmtTestMode = IfStmtTestMode.values()[i];
+      switch (ifStmtTestMode) {
+      case ROW_MODE:
+        doRowIfTest(
+            typeInfo, columns, children, randomRows, rowSource.rowStructObjectInspector(),
+            resultObjects);
+        break;
+      case ADAPTOR_WHEN:
+      case VECTOR_EXPRESSION:
+        doVectorIfTest(
+            typeInfo,
+            columns,
+            rowSource.typeInfos(),
+            rowSource.dataTypePhysicalVariations(),
+            children,
+            ifStmtTestMode,
+            columnScalarMode,
+            batchSource,
+            batchContext,
+            resultObjects);
+        break;
+      default:
+        throw new RuntimeException("Unexpected IF statement test mode " + ifStmtTestMode);
+      }
+    }
+
+    for (int i = 0; i < rowCount; i++) {
+      // Row-mode is the expected value.
+      Object expectedResult = resultObjectsArray[0][i];
+
+      for (int v = 1; v < IfStmtTestMode.count; v++) {
+        Object vectorResult = resultObjectsArray[v][i];
+        if (expectedResult == null || vectorResult == null) {
+          if (expectedResult != null || vectorResult != null) {
+            Assert.fail(
+                "Row " + i + " " + IfStmtTestMode.values()[v] +
+                " " + columnScalarMode +
+                " result is NULL " + (vectorResult == null) +
+                " does not match row-mode expected result is NULL " + (expectedResult == null));
+          }
+        } else {
+
+          if (isDecimal64 && expectedResult instanceof LongWritable) {
+
+            HiveDecimalWritable expectedHiveDecimalWritable = new HiveDecimalWritable(0);
+            expectedHiveDecimalWritable.deserialize64(
+                ((LongWritable) expectedResult).get(), decimal64Scale);
+            expectedResult = expectedHiveDecimalWritable;
+          }
+
+          if (!expectedResult.equals(vectorResult)) {
+            Assert.fail(
+                "Row " + i + " " + IfStmtTestMode.values()[v] +
+                " " + columnScalarMode +
+                " result " + vectorResult.toString() +
+                " (" + vectorResult.getClass().getSimpleName() + ")" +
+                " does not match row-mode expected result " + expectedResult.toString() +
+                " (" + expectedResult.getClass().getSimpleName() + ")");
+          }
+        }
+      }
+    }
+  }
+
+  private void doRowIfTest(TypeInfo typeInfo, List<String> columns, List<ExprNodeDesc> children,
+      Object[][] randomRows, ObjectInspector rowInspector, Object[] resultObjects) throws Exception {
+
+    GenericUDF udf = new GenericUDFIf();
+
+    ExprNodeGenericFuncDesc exprDesc =
+        new ExprNodeGenericFuncDesc(typeInfo, udf, children);
+    HiveConf hiveConf = new HiveConf();
+    ExprNodeEvaluator evaluator =
+        ExprNodeEvaluatorFactory.get(exprDesc, hiveConf);
+    evaluator.initialize(rowInspector);
+
+    final int rowCount = randomRows.length;
+    for (int i = 0; i < rowCount; i++) {
+      Object[] row = randomRows[i];
+      Object result = evaluator.evaluate(row);
+      resultObjects[i] = result;
+    }
+  }
+
+  private void extractResultObjects(VectorizedRowBatch batch, int rowIndex,
+      VectorExtractRow resultVectorExtractRow, Object[] scrqtchRow, Object[] resultObjects) {
+    // UNDONE: selectedInUse
+    for (int i = 0; i < batch.size; i++) {
+      resultVectorExtractRow.extractRow(batch, i, scrqtchRow);
+
+      // UNDONE: Need to copy the object.
+      resultObjects[rowIndex++] = scrqtchRow[0];
+    }
+  }
+
+  private void doVectorIfTest(TypeInfo typeInfo,
+      List<String> columns,
+      TypeInfo[] typeInfos, DataTypePhysicalVariation[] dataTypePhysicalVariations,
+      List<ExprNodeDesc> children,
+      IfStmtTestMode ifStmtTestMode, ColumnScalarMode columnScalarMode,
+      VectorRandomBatchSource batchSource, VectorizedRowBatchCtx batchContext,
+      Object[] resultObjects)
+          throws Exception {
+
+    GenericUDF udf;
+    switch (ifStmtTestMode) {
+    case VECTOR_EXPRESSION:
+      udf = new GenericUDFIf();
+      break;
+    case ADAPTOR_WHEN:
+      udf = new GenericUDFWhen();
+      break;
+    default:
+      throw new RuntimeException("Unexpected IF statement test mode " + ifStmtTestMode);
+    }
+
+    ExprNodeGenericFuncDesc exprDesc =
+        new ExprNodeGenericFuncDesc(typeInfo, udf, children);
+
+    String ifExprMode = (ifStmtTestMode != IfStmtTestMode.VECTOR_EXPRESSION ? "adaptor" : "good");
+    HiveConf hiveConf = new HiveConf();
+    hiveConf.setVar(HiveConf.ConfVars.HIVE_VECTORIZED_IF_EXPR_MODE, ifExprMode);
+
+    VectorizationContext vectorizationContext =
+        new VectorizationContext(
+            "name",
+            columns,
+            Arrays.asList(typeInfos),
+            Arrays.asList(dataTypePhysicalVariations),
+            hiveConf);
+    VectorExpression vectorExpression = vectorizationContext.getVectorExpression(exprDesc);
+
+    VectorizedRowBatch batch = batchContext.createVectorizedRowBatch();
+
+    VectorExtractRow resultVectorExtractRow = new VectorExtractRow();
+    resultVectorExtractRow.init(new TypeInfo[] { typeInfo }, new int[] { columns.size() });
+    Object[] scrqtchRow = new Object[1];
+
+    /*
+    System.out.println(
+        "*DEBUG* typeInfo " + typeInfo.toString() +
+        " ifStmtTestMode " + ifStmtTestMode +
+        " columnScalarMode " + columnScalarMode +
+        " vectorExpression " + vectorExpression.getClass().getSimpleName());
+    */
+
+    batchSource.resetBatchIteration();
+    int rowIndex = 0;
+    while (true) {
+      if (!batchSource.fillNextBatch(batch)) {
+        break;
+      }
+      vectorExpression.evaluate(batch);
+      extractResultObjects(batch, rowIndex, resultVectorExtractRow, scrqtchRow, resultObjects);
+      rowIndex += batch.size;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/e2d4f347/ql/src/test/queries/clientpositive/vectorized_timestamp_funcs.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/vectorized_timestamp_funcs.q b/ql/src/test/queries/clientpositive/vectorized_timestamp_funcs.q
index afbc18a..859289c 100644
--- a/ql/src/test/queries/clientpositive/vectorized_timestamp_funcs.q
+++ b/ql/src/test/queries/clientpositive/vectorized_timestamp_funcs.q
@@ -1,22 +1,40 @@
 set hive.mapred.mode=nonstrict;
 set hive.explain.user=false;
 set hive.fetch.task.conversion=none;
+SET hive.vectorized.execution.enabled = false;
+
 -- Test timestamp functions in vectorized mode to verify they run correctly end-to-end.
 -- Turning on vectorization has been temporarily moved after filling the test table
 -- due to bug HIVE-8197.
 
+-- SORT_QUERY_RESULTS
 
-CREATE TABLE alltypesorc_string(ctimestamp1 timestamp, stimestamp1 string) STORED AS ORC;
+CREATE TABLE alltypesorc_string(cboolean1 boolean, ctimestamp1 timestamp, stimestamp1 string,
+    ctimestamp2 timestamp) STORED AS ORC;
 
 INSERT OVERWRITE TABLE alltypesorc_string
 SELECT
+  cboolean1,
   to_utc_timestamp(ctimestamp1, 'America/Los_Angeles') AS toutc,
-  CAST(to_utc_timestamp(ctimestamp1, 'America/Los_Angeles') AS STRING) as cst
+  CAST(to_utc_timestamp(ctimestamp1, 'America/Los_Angeles') AS STRING) as cst,
+  ctimestamp2
 FROM alltypesorc
 ORDER BY toutc, cst
 LIMIT 40;
-
-SET hive.vectorized.execution.enabled = true;
+INSERT INTO TABLE alltypesorc_string values (false, '2021-09-24 03:18:32.4', '1978-08-05 14:41:05.501', '1999-10-03 16:59:10.396903939');
+INSERT INTO TABLE alltypesorc_string values (false, null, '2013-04-10 00:43:46.8547315', null);
+INSERT INTO TABLE alltypesorc_string values (false, '2021-09-24 03:18:32.4', null, null);
+INSERT INTO TABLE alltypesorc_string values (null, '7160-12-02 06:00:24.81200852', '0004-09-22 18:26:29.519542222', '1966-08-16 13:36:50.183');
+INSERT INTO TABLE alltypesorc_string values (null, null, '4966-12-04 09:30:55.202', null);
+INSERT INTO TABLE alltypesorc_string values (null, '7160-12-02 06:00:24.81200852', null, null);
+INSERT INTO TABLE alltypesorc_string values (true, '1985-07-20 09:30:11.0', '8521-01-16 20:42:05.668832', '1319-02-02 16:31:57.778');
+INSERT INTO TABLE alltypesorc_string values (true, null, '1883-04-17 04:14:34.64776', '2024-11-11 16:42:41.101');
+INSERT INTO TABLE alltypesorc_string values (true, '0528-10-27 08:15:18.941718273', null, null);
+
+INSERT INTO TABLE alltypesorc_string values
+     (false, '2021-09-24 03:18:32.4', '1985-11-18 16:37:54.0', '2010-04-08 02:43:35.861742727'),
+     (true, null, '1985-11-18 16:37:54.0', null),
+     (null, '2021-09-24 03:18:32.4', null, '1974-10-04 17:21:03.989');
 
 CREATE TABLE alltypesorc_wrong(stimestamp1 string) STORED AS ORC;
 
@@ -24,6 +42,8 @@ INSERT INTO TABLE alltypesorc_wrong SELECT 'abcd' FROM alltypesorc LIMIT 1;
 INSERT INTO TABLE alltypesorc_wrong SELECT '2000:01:01 00-00-00' FROM alltypesorc LIMIT 1;
 INSERT INTO TABLE alltypesorc_wrong SELECT '0000-00-00 99:99:99' FROM alltypesorc LIMIT 1;
 
+SET hive.vectorized.execution.enabled = true;
+
 EXPLAIN VECTORIZATION EXPRESSION  SELECT
   to_unix_timestamp(ctimestamp1) AS c1,
   year(ctimestamp1),
@@ -33,7 +53,15 @@ EXPLAIN VECTORIZATION EXPRESSION  SELECT
   weekofyear(ctimestamp1),
   hour(ctimestamp1),
   minute(ctimestamp1),
-  second(ctimestamp1)
+  second(ctimestamp1),
+  cboolean1,
+  ctimestamp1,
+  ctimestamp2,
+  if (cboolean1, ctimestamp1, timestamp '1319-02-02 16:31:57.778'),
+  if (cboolean1, timestamp '2000-12-18 08:42:30.0005', ctimestamp1),
+  if (cboolean1, ctimestamp1, ctimestamp2),
+  if (cboolean1, ctimestamp1, null),
+  if (cboolean1, null, ctimestamp2)
 FROM alltypesorc_string
 ORDER BY c1;
 
@@ -46,7 +74,15 @@ SELECT
   weekofyear(ctimestamp1),
   hour(ctimestamp1),
   minute(ctimestamp1),
-  second(ctimestamp1)
+  second(ctimestamp1),
+  cboolean1,
+  ctimestamp1,
+  ctimestamp2,
+  if (cboolean1, ctimestamp1, timestamp '1319-02-02 16:31:57.778'),
+  if (cboolean1, timestamp '2000-12-18 08:42:30.0005', ctimestamp1),
+  if (cboolean1, ctimestamp1, ctimestamp2),
+  if (cboolean1, ctimestamp1, null),
+  if (cboolean1, null, ctimestamp2)
 FROM alltypesorc_string
 ORDER BY c1;
 

http://git-wip-us.apache.org/repos/asf/hive/blob/e2d4f347/ql/src/test/results/clientpositive/llap/vectorized_timestamp_funcs.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/vectorized_timestamp_funcs.q.out b/ql/src/test/results/clientpositive/llap/vectorized_timestamp_funcs.q.out
index c9dd434..79ba4c6 100644
--- a/ql/src/test/results/clientpositive/llap/vectorized_timestamp_funcs.q.out
+++ b/ql/src/test/results/clientpositive/llap/vectorized_timestamp_funcs.q.out
@@ -1,15 +1,19 @@
-PREHOOK: query: CREATE TABLE alltypesorc_string(ctimestamp1 timestamp, stimestamp1 string) STORED AS ORC
+PREHOOK: query: CREATE TABLE alltypesorc_string(cboolean1 boolean, ctimestamp1 timestamp, stimestamp1 string,
+    ctimestamp2 timestamp) STORED AS ORC
 PREHOOK: type: CREATETABLE
 PREHOOK: Output: database:default
 PREHOOK: Output: default@alltypesorc_string
-POSTHOOK: query: CREATE TABLE alltypesorc_string(ctimestamp1 timestamp, stimestamp1 string) STORED AS ORC
+POSTHOOK: query: CREATE TABLE alltypesorc_string(cboolean1 boolean, ctimestamp1 timestamp, stimestamp1 string,
+    ctimestamp2 timestamp) STORED AS ORC
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@alltypesorc_string
 PREHOOK: query: INSERT OVERWRITE TABLE alltypesorc_string
 SELECT
+  cboolean1,
   to_utc_timestamp(ctimestamp1, 'America/Los_Angeles') AS toutc,
-  CAST(to_utc_timestamp(ctimestamp1, 'America/Los_Angeles') AS STRING) as cst
+  CAST(to_utc_timestamp(ctimestamp1, 'America/Los_Angeles') AS STRING) as cst,
+  ctimestamp2
 FROM alltypesorc
 ORDER BY toutc, cst
 LIMIT 40
@@ -18,16 +22,146 @@ PREHOOK: Input: default@alltypesorc
 PREHOOK: Output: default@alltypesorc_string
 POSTHOOK: query: INSERT OVERWRITE TABLE alltypesorc_string
 SELECT
+  cboolean1,
   to_utc_timestamp(ctimestamp1, 'America/Los_Angeles') AS toutc,
-  CAST(to_utc_timestamp(ctimestamp1, 'America/Los_Angeles') AS STRING) as cst
+  CAST(to_utc_timestamp(ctimestamp1, 'America/Los_Angeles') AS STRING) as cst,
+  ctimestamp2
 FROM alltypesorc
 ORDER BY toutc, cst
 LIMIT 40
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@alltypesorc
 POSTHOOK: Output: default@alltypesorc_string
+POSTHOOK: Lineage: alltypesorc_string.cboolean1 SIMPLE [(alltypesorc)alltypesorc.FieldSchema(name:cboolean1, type:boolean, comment:null), ]
 POSTHOOK: Lineage: alltypesorc_string.ctimestamp1 EXPRESSION [(alltypesorc)alltypesorc.FieldSchema(name:ctimestamp1, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: alltypesorc_string.ctimestamp2 SIMPLE [(alltypesorc)alltypesorc.FieldSchema(name:ctimestamp2, type:timestamp, comment:null), ]
 POSTHOOK: Lineage: alltypesorc_string.stimestamp1 EXPRESSION [(alltypesorc)alltypesorc.FieldSchema(name:ctimestamp1, type:timestamp, comment:null), ]
+PREHOOK: query: INSERT INTO TABLE alltypesorc_string values (false, '2021-09-24 03:18:32.4', '1978-08-05 14:41:05.501', '1999-10-03 16:59:10.396903939')
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@alltypesorc_string
+POSTHOOK: query: INSERT INTO TABLE alltypesorc_string values (false, '2021-09-24 03:18:32.4', '1978-08-05 14:41:05.501', '1999-10-03 16:59:10.396903939')
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@alltypesorc_string
+POSTHOOK: Lineage: alltypesorc_string.cboolean1 SCRIPT []
+POSTHOOK: Lineage: alltypesorc_string.ctimestamp1 SCRIPT []
+POSTHOOK: Lineage: alltypesorc_string.ctimestamp2 SCRIPT []
+POSTHOOK: Lineage: alltypesorc_string.stimestamp1 SCRIPT []
+PREHOOK: query: INSERT INTO TABLE alltypesorc_string values (false, null, '2013-04-10 00:43:46.8547315', null)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@alltypesorc_string
+POSTHOOK: query: INSERT INTO TABLE alltypesorc_string values (false, null, '2013-04-10 00:43:46.8547315', null)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@alltypesorc_string
+POSTHOOK: Lineage: alltypesorc_string.cboolean1 SCRIPT []
+POSTHOOK: Lineage: alltypesorc_string.ctimestamp1 EXPRESSION []
+POSTHOOK: Lineage: alltypesorc_string.ctimestamp2 EXPRESSION []
+POSTHOOK: Lineage: alltypesorc_string.stimestamp1 SCRIPT []
+PREHOOK: query: INSERT INTO TABLE alltypesorc_string values (false, '2021-09-24 03:18:32.4', null, null)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@alltypesorc_string
+POSTHOOK: query: INSERT INTO TABLE alltypesorc_string values (false, '2021-09-24 03:18:32.4', null, null)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@alltypesorc_string
+POSTHOOK: Lineage: alltypesorc_string.cboolean1 SCRIPT []
+POSTHOOK: Lineage: alltypesorc_string.ctimestamp1 SCRIPT []
+POSTHOOK: Lineage: alltypesorc_string.ctimestamp2 EXPRESSION []
+POSTHOOK: Lineage: alltypesorc_string.stimestamp1 EXPRESSION []
+PREHOOK: query: INSERT INTO TABLE alltypesorc_string values (null, '7160-12-02 06:00:24.81200852', '0004-09-22 18:26:29.519542222', '1966-08-16 13:36:50.183')
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@alltypesorc_string
+POSTHOOK: query: INSERT INTO TABLE alltypesorc_string values (null, '7160-12-02 06:00:24.81200852', '0004-09-22 18:26:29.519542222', '1966-08-16 13:36:50.183')
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@alltypesorc_string
+POSTHOOK: Lineage: alltypesorc_string.cboolean1 EXPRESSION []
+POSTHOOK: Lineage: alltypesorc_string.ctimestamp1 SCRIPT []
+POSTHOOK: Lineage: alltypesorc_string.ctimestamp2 SCRIPT []
+POSTHOOK: Lineage: alltypesorc_string.stimestamp1 SCRIPT []
+PREHOOK: query: INSERT INTO TABLE alltypesorc_string values (null, null, '4966-12-04 09:30:55.202', null)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@alltypesorc_string
+POSTHOOK: query: INSERT INTO TABLE alltypesorc_string values (null, null, '4966-12-04 09:30:55.202', null)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@alltypesorc_string
+POSTHOOK: Lineage: alltypesorc_string.cboolean1 EXPRESSION []
+POSTHOOK: Lineage: alltypesorc_string.ctimestamp1 EXPRESSION []
+POSTHOOK: Lineage: alltypesorc_string.ctimestamp2 EXPRESSION []
+POSTHOOK: Lineage: alltypesorc_string.stimestamp1 SCRIPT []
+PREHOOK: query: INSERT INTO TABLE alltypesorc_string values (null, '7160-12-02 06:00:24.81200852', null, null)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@alltypesorc_string
+POSTHOOK: query: INSERT INTO TABLE alltypesorc_string values (null, '7160-12-02 06:00:24.81200852', null, null)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@alltypesorc_string
+POSTHOOK: Lineage: alltypesorc_string.cboolean1 EXPRESSION []
+POSTHOOK: Lineage: alltypesorc_string.ctimestamp1 SCRIPT []
+POSTHOOK: Lineage: alltypesorc_string.ctimestamp2 EXPRESSION []
+POSTHOOK: Lineage: alltypesorc_string.stimestamp1 EXPRESSION []
+PREHOOK: query: INSERT INTO TABLE alltypesorc_string values (true, '1985-07-20 09:30:11.0', '8521-01-16 20:42:05.668832', '1319-02-02 16:31:57.778')
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@alltypesorc_string
+POSTHOOK: query: INSERT INTO TABLE alltypesorc_string values (true, '1985-07-20 09:30:11.0', '8521-01-16 20:42:05.668832', '1319-02-02 16:31:57.778')
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@alltypesorc_string
+POSTHOOK: Lineage: alltypesorc_string.cboolean1 SCRIPT []
+POSTHOOK: Lineage: alltypesorc_string.ctimestamp1 SCRIPT []
+POSTHOOK: Lineage: alltypesorc_string.ctimestamp2 SCRIPT []
+POSTHOOK: Lineage: alltypesorc_string.stimestamp1 SCRIPT []
+PREHOOK: query: INSERT INTO TABLE alltypesorc_string values (true, null, '1883-04-17 04:14:34.64776', '2024-11-11 16:42:41.101')
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@alltypesorc_string
+POSTHOOK: query: INSERT INTO TABLE alltypesorc_string values (true, null, '1883-04-17 04:14:34.64776', '2024-11-11 16:42:41.101')
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@alltypesorc_string
+POSTHOOK: Lineage: alltypesorc_string.cboolean1 SCRIPT []
+POSTHOOK: Lineage: alltypesorc_string.ctimestamp1 EXPRESSION []
+POSTHOOK: Lineage: alltypesorc_string.ctimestamp2 SCRIPT []
+POSTHOOK: Lineage: alltypesorc_string.stimestamp1 SCRIPT []
+PREHOOK: query: INSERT INTO TABLE alltypesorc_string values (true, '0528-10-27 08:15:18.941718273', null, null)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@alltypesorc_string
+POSTHOOK: query: INSERT INTO TABLE alltypesorc_string values (true, '0528-10-27 08:15:18.941718273', null, null)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@alltypesorc_string
+POSTHOOK: Lineage: alltypesorc_string.cboolean1 SCRIPT []
+POSTHOOK: Lineage: alltypesorc_string.ctimestamp1 SCRIPT []
+POSTHOOK: Lineage: alltypesorc_string.ctimestamp2 EXPRESSION []
+POSTHOOK: Lineage: alltypesorc_string.stimestamp1 EXPRESSION []
+PREHOOK: query: INSERT INTO TABLE alltypesorc_string values
+     (false, '2021-09-24 03:18:32.4', '1985-11-18 16:37:54.0', '2010-04-08 02:43:35.861742727'),
+     (true, null, '1985-11-18 16:37:54.0', null),
+     (null, '2021-09-24 03:18:32.4', null, '1974-10-04 17:21:03.989')
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@alltypesorc_string
+POSTHOOK: query: INSERT INTO TABLE alltypesorc_string values
+     (false, '2021-09-24 03:18:32.4', '1985-11-18 16:37:54.0', '2010-04-08 02:43:35.861742727'),
+     (true, null, '1985-11-18 16:37:54.0', null),
+     (null, '2021-09-24 03:18:32.4', null, '1974-10-04 17:21:03.989')
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@alltypesorc_string
+POSTHOOK: Lineage: alltypesorc_string.cboolean1 SCRIPT []
+POSTHOOK: Lineage: alltypesorc_string.ctimestamp1 SCRIPT []
+POSTHOOK: Lineage: alltypesorc_string.ctimestamp2 SCRIPT []
+POSTHOOK: Lineage: alltypesorc_string.stimestamp1 SCRIPT []
 PREHOOK: query: CREATE TABLE alltypesorc_wrong(stimestamp1 string) STORED AS ORC
 PREHOOK: type: CREATETABLE
 PREHOOK: Output: database:default
@@ -72,7 +206,15 @@ PREHOOK: query: EXPLAIN VECTORIZATION EXPRESSION  SELECT
   weekofyear(ctimestamp1),
   hour(ctimestamp1),
   minute(ctimestamp1),
-  second(ctimestamp1)
+  second(ctimestamp1),
+  cboolean1,
+  ctimestamp1,
+  ctimestamp2,
+  if (cboolean1, ctimestamp1, timestamp '1319-02-02 16:31:57.778'),
+  if (cboolean1, timestamp '2000-12-18 08:42:30.0005', ctimestamp1),
+  if (cboolean1, ctimestamp1, ctimestamp2),
+  if (cboolean1, ctimestamp1, null),
+  if (cboolean1, null, ctimestamp2)
 FROM alltypesorc_string
 ORDER BY c1
 PREHOOK: type: QUERY
@@ -85,7 +227,15 @@ POSTHOOK: query: EXPLAIN VECTORIZATION EXPRESSION  SELECT
   weekofyear(ctimestamp1),
   hour(ctimestamp1),
   minute(ctimestamp1),
-  second(ctimestamp1)
+  second(ctimestamp1),
+  cboolean1,
+  ctimestamp1,
+  ctimestamp2,
+  if (cboolean1, ctimestamp1, timestamp '1319-02-02 16:31:57.778'),
+  if (cboolean1, timestamp '2000-12-18 08:42:30.0005', ctimestamp1),
+  if (cboolean1, ctimestamp1, ctimestamp2),
+  if (cboolean1, ctimestamp1, null),
+  if (cboolean1, null, ctimestamp2)
 FROM alltypesorc_string
 ORDER BY c1
 POSTHOOK: type: QUERY
@@ -109,18 +259,18 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: alltypesorc_string
-                  Statistics: Num rows: 40 Data size: 1600 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 52 Data size: 4356 Basic stats: COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
                   Select Operator
-                    expressions: to_unix_timestamp(ctimestamp1) (type: bigint), year(ctimestamp1) (type: int), month(ctimestamp1) (type: int), day(ctimestamp1) (type: int), dayofmonth(ctimestamp1) (type: int), weekofyear(ctimestamp1) (type: int), hour(ctimestamp1) (type: int), minute(ctimestamp1) (type: int), second(ctimestamp1) (type: int)
-                    outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
+                    expressions: to_unix_timestamp(ctimestamp1) (type: bigint), year(ctimestamp1) (type: int), month(ctimestamp1) (type: int), day(ctimestamp1) (type: int), dayofmonth(ctimestamp1) (type: int), weekofyear(ctimestamp1) (type: int), hour(ctimestamp1) (type: int), minute(ctimestamp1) (type: int), second(ctimestamp1) (type: int), cboolean1 (type: boolean), ctimestamp1 (type: timestamp), ctimestamp2 (type: timestamp), if(cboolean1, ctimestamp1, TIMESTAMP'1319-02-02 16:31:57.778') (type: timestamp), if(cboolean1, TIMESTAMP'2000-12-18 08:42:30.0005', ctimestamp1) (type: timestamp), if(cboolean1, ctimestamp1, ctimestamp2) (type: timestamp), if(cboolean1, ctimestamp1, null) (type: timestamp), if(cboolean1, null, ctimestamp2) (type: timestamp)
+                    outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16
                     Select Vectorization:
                         className: VectorSelectOperator
                         native: true
-                        projectedOutputColumnNums: [3, 4, 5, 6, 7, 8, 9, 10, 11]
-                        selectExpressions: VectorUDFUnixTimeStampTimestamp(col 0:timestamp) -> 3:bigint, VectorUDFYearTimestamp(col 0:timestamp, field YEAR) -> 4:int, VectorUDFMonthTimestamp(col 0:timestamp, field MONTH) -> 5:int, VectorUDFDayOfMonthTimestamp(col 0:timestamp, field DAY_OF_MONTH) -> 6:int, VectorUDFDayOfMonthTimestamp(col 0:timestamp, field DAY_OF_MONTH) -> 7:int, VectorUDFWeekOfYearTimestamp(col 0:timestamp, field WEEK_OF_YEAR) -> 8:int, VectorUDFHourTimestamp(col 0:timestamp, field HOUR_OF_DAY) -> 9:int, VectorUDFMinuteTimestamp(col 0:timestamp, field MINUTE) -> 10:int, VectorUDFSecondTimestamp(col 0:timestamp, field SECOND) -> 11:int
-                    Statistics: Num rows: 40 Data size: 1600 Basic stats: COMPLETE Column stats: COMPLETE
+                        projectedOutputColumnNums: [5, 6, 7, 8, 9, 10, 11, 12, 13, 0, 1, 3, 14, 15, 16, 17, 18]
+                        selectExpressions: VectorUDFUnixTimeStampTimestamp(col 1:timestamp) -> 5:bigint, VectorUDFYearTimestamp(col 1:timestamp, field YEAR) -> 6:int, VectorUDFMonthTimestamp(col 1:timestamp, field MONTH) -> 7:int, VectorUDFDayOfMonthTimestamp(col 1:timestamp, field DAY_OF_MONTH) -> 8:int, VectorUDFDayOfMonthTimestamp(col 1:timestamp, field DAY_OF_MONTH) -> 9:int, VectorUDFWeekOfYearTimestamp(col 1:timestamp, field WEEK_OF_YEAR) -> 10:int, VectorUDFHourTimestamp(col 1:timestamp, field HOUR_OF_DAY) -> 11:int, VectorUDFMinuteTimestamp(col 1:timestamp, field MINUTE) -> 12:int, VectorUDFSecondTimestamp(col 1:timestamp, field SECOND) -> 13:int, IfExprTimestampColumnScalar(col 0:boolean, col 1:timestamp, val 1319-02-02 16:31:57.778) -> 14:timestamp, IfExprTimestampScalarColumn(col 0:boolean, val 2000-12-18 08:42:30.0005, col 1:timestamp) -> 15:timestamp, IfExprTimestampColumnColumn(col 0:boolean, col 1:timestampcol 3:timestamp) -> 16:timestamp, IfExprColumnNull(col 0:boole
 an, col 1:timestamp, null)(children: col 0:boolean, col 1:timestamp) -> 17:timestamp, IfExprNullColumn(col 0:boolean, null, col 3)(children: col 0:boolean, col 3:timestamp) -> 18:timestamp
+                    Statistics: Num rows: 52 Data size: 16836 Basic stats: COMPLETE Column stats: COMPLETE
                     Reduce Output Operator
                       key expressions: _col0 (type: bigint)
                       sort order: +
@@ -128,8 +278,8 @@ STAGE PLANS:
                           className: VectorReduceSinkObjectHashOperator
                           native: true
                           nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true
-                      Statistics: Num rows: 40 Data size: 1600 Basic stats: COMPLETE Column stats: COMPLETE
-                      value expressions: _col1 (type: int), _col2 (type: int), _col3 (type: int), _col4 (type: int), _col5 (type: int), _col6 (type: int), _col7 (type: int), _col8 (type: int)
+                      Statistics: Num rows: 52 Data size: 16836 Basic stats: COMPLETE Column stats: COMPLETE
+                      value expressions: _col1 (type: int), _col2 (type: int), _col3 (type: int), _col4 (type: int), _col5 (type: int), _col6 (type: int), _col7 (type: int), _col8 (type: int), _col9 (type: boolean), _col10 (type: timestamp), _col11 (type: timestamp), _col12 (type: timestamp), _col13 (type: timestamp), _col14 (type: timestamp), _col15 (type: timestamp), _col16 (type: timestamp)
             Execution mode: vectorized, llap
             LLAP IO: all inputs
             Map Vectorization:
@@ -151,19 +301,19 @@ STAGE PLANS:
                 vectorized: true
             Reduce Operator Tree:
               Select Operator
-                expressions: KEY.reducesinkkey0 (type: bigint), VALUE._col0 (type: int), VALUE._col1 (type: int), VALUE._col2 (type: int), VALUE._col3 (type: int), VALUE._col4 (type: int), VALUE._col5 (type: int), VALUE._col6 (type: int), VALUE._col7 (type: int)
-                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
+                expressions: KEY.reducesinkkey0 (type: bigint), VALUE._col0 (type: int), VALUE._col1 (type: int), VALUE._col2 (type: int), VALUE._col3 (type: int), VALUE._col4 (type: int), VALUE._col5 (type: int), VALUE._col6 (type: int), VALUE._col7 (type: int), VALUE._col8 (type: boolean), VALUE._col9 (type: timestamp), VALUE._col10 (type: timestamp), VALUE._col11 (type: timestamp), VALUE._col12 (type: timestamp), VALUE._col13 (type: timestamp), VALUE._col14 (type: timestamp), VALUE._col15 (type: timestamp)
+                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16
                 Select Vectorization:
                     className: VectorSelectOperator
                     native: true
-                    projectedOutputColumnNums: [0, 1, 2, 3, 4, 5, 6, 7, 8]
-                Statistics: Num rows: 40 Data size: 1600 Basic stats: COMPLETE Column stats: COMPLETE
+                    projectedOutputColumnNums: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16]
+                Statistics: Num rows: 52 Data size: 16836 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
                   File Sink Vectorization:
                       className: VectorFileSinkOperator
                       native: false
-                  Statistics: Num rows: 40 Data size: 1600 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 52 Data size: 16836 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -184,7 +334,15 @@ PREHOOK: query: SELECT
   weekofyear(ctimestamp1),
   hour(ctimestamp1),
   minute(ctimestamp1),
-  second(ctimestamp1)
+  second(ctimestamp1),
+  cboolean1,
+  ctimestamp1,
+  ctimestamp2,
+  if (cboolean1, ctimestamp1, timestamp '1319-02-02 16:31:57.778'),
+  if (cboolean1, timestamp '2000-12-18 08:42:30.0005', ctimestamp1),
+  if (cboolean1, ctimestamp1, ctimestamp2),
+  if (cboolean1, ctimestamp1, null),
+  if (cboolean1, null, ctimestamp2)
 FROM alltypesorc_string
 ORDER BY c1
 PREHOOK: type: QUERY
@@ -199,52 +357,72 @@ POSTHOOK: query: SELECT
   weekofyear(ctimestamp1),
   hour(ctimestamp1),
   minute(ctimestamp1),
-  second(ctimestamp1)
+  second(ctimestamp1),
+  cboolean1,
+  ctimestamp1,
+  ctimestamp2,
+  if (cboolean1, ctimestamp1, timestamp '1319-02-02 16:31:57.778'),
+  if (cboolean1, timestamp '2000-12-18 08:42:30.0005', ctimestamp1),
+  if (cboolean1, ctimestamp1, ctimestamp2),
+  if (cboolean1, ctimestamp1, null),
+  if (cboolean1, null, ctimestamp2)
 FROM alltypesorc_string
 ORDER BY c1
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@alltypesorc_string
 #### A masked pattern was here ####
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
+-45479000681	528	10	27	27	43	8	15	18	true	0528-10-27 08:15:18.941718273	NULL	0528-10-27 08:15:18.941718273	2000-12-18 08:42:30.0005	0528-10-27 08:15:18.941718273	0528-10-27 08:15:18.941718273	NULL
+1632478712	2021	9	24	24	38	3	18	32	NULL	2021-09-24 03:18:32.4	1974-10-04 17:21:03.989	1319-02-02 16:31:57.778	2021-09-24 03:18:32.4	1974-10-04 17:21:03.989	NULL	1974-10-04 17:21:03.989
+1632478712	2021	9	24	24	38	3	18	32	false	2021-09-24 03:18:32.4	1999-10-03 16:59:10.396903939	1319-02-02 16:31:57.778	2021-09-24 03:18:32.4	1999-10-03 16:59:10.396903939	NULL	1999-10-03 16:59:10.396903939
+1632478712	2021	9	24	24	38	3	18	32	false	2021-09-24 03:18:32.4	2010-04-08 02:43:35.861742727	1319-02-02 16:31:57.778	2021-09-24 03:18:32.4	2010-04-08 02:43:35.861742727	NULL	2010-04-08 02:43:35.861742727
+1632478712	2021	9	24	24	38	3	18	32	false	2021-09-24 03:18:32.4	NULL	1319-02-02 16:31:57.778	2021-09-24 03:18:32.4	NULL	NULL	NULL
+163809612024	7160	12	2	2	48	6	0	24	NULL	7160-12-02 06:00:24.81200852	1966-08-16 13:36:50.183	1319-02-02 16:31:57.778	7160-12-02 06:00:24.81200852	1966-08-16 13:36:50.183	NULL	1966-08-16 13:36:50.183
+163809612024	7160	12	2	2	48	6	0	24	NULL	7160-12-02 06:00:24.81200852	NULL	1319-02-02 16:31:57.778	7160-12-02 06:00:24.81200852	NULL	NULL	NULL
+490725011	1985	7	20	20	29	9	30	11	true	1985-07-20 09:30:11	1319-02-02 16:31:57.778	1985-07-20 09:30:11	2000-12-18 08:42:30.0005	1985-07-20 09:30:11	1985-07-20 09:30:11	NULL
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	1319-02-02 16:31:57.778	NULL	NULL	NULL	NULL
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	false	NULL	1969-12-31 15:59:44.028	1319-02-02 16:31:57.778	NULL	1969-12-31 15:59:44.028	NULL	1969-12-31 15:59:44.028
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	false	NULL	1969-12-31 15:59:44.809	1319-02-02 16:31:57.778	NULL	1969-12-31 15:59:44.809	NULL	1969-12-31 15:59:44.809
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	false	NULL	1969-12-31 15:59:50.531	1319-02-02 16:31:57.778	NULL	1969-12-31 15:59:50.531	NULL	1969-12-31 15:59:50.531
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	false	NULL	1969-12-31 15:59:51.009	1319-02-02 16:31:57.778	NULL	1969-12-31 15:59:51.009	NULL	1969-12-31 15:59:51.009
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	false	NULL	1969-12-31 15:59:53.761	1319-02-02 16:31:57.778	NULL	1969-12-31 15:59:53.761	NULL	1969-12-31 15:59:53.761
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	false	NULL	1969-12-31 16:00:00.905	1319-02-02 16:31:57.778	NULL	1969-12-31 16:00:00.905	NULL	1969-12-31 16:00:00.905
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	false	NULL	1969-12-31 16:00:03.586	1319-02-02 16:31:57.778	NULL	1969-12-31 16:00:03.586	NULL	1969-12-31 16:00:03.586
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	false	NULL	1969-12-31 16:00:05.227	1319-02-02 16:31:57.778	NULL	1969-12-31 16:00:05.227	NULL	1969-12-31 16:00:05.227
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	false	NULL	1969-12-31 16:00:05.535	1319-02-02 16:31:57.778	NULL	1969-12-31 16:00:05.535	NULL	1969-12-31 16:00:05.535
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	false	NULL	1969-12-31 16:00:07.02	1319-02-02 16:31:57.778	NULL	1969-12-31 16:00:07.02	NULL	1969-12-31 16:00:07.02
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	false	NULL	1969-12-31 16:00:07.365	1319-02-02 16:31:57.778	NULL	1969-12-31 16:00:07.365	NULL	1969-12-31 16:00:07.365
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	false	NULL	1969-12-31 16:00:07.517	1319-02-02 16:31:57.778	NULL	1969-12-31 16:00:07.517	NULL	1969-12-31 16:00:07.517
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	false	NULL	1969-12-31 16:00:07.767	1319-02-02 16:31:57.778	NULL	1969-12-31 16:00:07.767	NULL	1969-12-31 16:00:07.767
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	false	NULL	1969-12-31 16:00:08.602	1319-02-02 16:31:57.778	NULL	1969-12-31 16:00:08.602	NULL	1969-12-31 16:00:08.602
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	false	NULL	1969-12-31 16:00:09.938	1319-02-02 16:31:57.778	NULL	1969-12-31 16:00:09.938	NULL	1969-12-31 16:00:09.938
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	false	NULL	1969-12-31 16:00:14.214	1319-02-02 16:31:57.778	NULL	1969-12-31 16:00:14.214	NULL	1969-12-31 16:00:14.214
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	false	NULL	1969-12-31 16:00:14.783	1319-02-02 16:31:57.778	NULL	1969-12-31 16:00:14.783	NULL	1969-12-31 16:00:14.783
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	false	NULL	NULL	1319-02-02 16:31:57.778	NULL	NULL	NULL	NULL
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	false	NULL	NULL	1319-02-02 16:31:57.778	NULL	NULL	NULL	NULL
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	true	NULL	1969-12-31 15:59:43.773	NULL	2000-12-18 08:42:30.0005	NULL	NULL	NULL
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	true	NULL	1969-12-31 15:59:44.262	NULL	2000-12-18 08:42:30.0005	NULL	NULL	NULL
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	true	NULL	1969-12-31 15:59:44.568	NULL	2000-12-18 08:42:30.0005	NULL	NULL	NULL
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	true	NULL	1969-12-31 15:59:47.351	NULL	2000-12-18 08:42:30.0005	NULL	NULL	NULL
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	true	NULL	1969-12-31 15:59:47.446	NULL	2000-12-18 08:42:30.0005	NULL	NULL	NULL
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	true	NULL	1969-12-31 15:59:48.023	NULL	2000-12-18 08:42:30.0005	NULL	NULL	NULL
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	true	NULL	1969-12-31 15:59:48.629	NULL	2000-12-18 08:42:30.0005	NULL	NULL	NULL
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	true	NULL	1969-12-31 15:59:49.177	NULL	2000-12-18 08:42:30.0005	NULL	NULL	NULL
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	true	NULL	1969-12-31 15:59:49.208	NULL	2000-12-18 08:42:30.0005	NULL	NULL	NULL
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	true	NULL	1969-12-31 15:59:50.789	NULL	2000-12-18 08:42:30.0005	NULL	NULL	NULL
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	true	NULL	1969-12-31 15:59:51.245	NULL	2000-12-18 08:42:30.0005	NULL	NULL	NULL
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	true	NULL	1969-12-31 15:59:52.372	NULL	2000-12-18 08:42:30.0005	NULL	NULL	NULL
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	true	NULL	1969-12-31 15:59:55.249	NULL	2000-12-18 08:42:30.0005	NULL	NULL	NULL
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	true	NULL	1969-12-31 16:00:00.661	NULL	2000-12-18 08:42:30.0005	NULL	NULL	NULL
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	true	NULL	1969-12-31 16:00:00.784	NULL	2000-12-18 08:42:30.0005	NULL	NULL	NULL
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	true	NULL	1969-12-31 16:00:01.836	NULL	2000-12-18 08:42:30.0005	NULL	NULL	NULL
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	true	NULL	1969-12-31 16:00:09.313	NULL	2000-12-18 08:42:30.0005	NULL	NULL	NULL
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	true	NULL	1969-12-31 16:00:09.538	NULL	2000-12-18 08:42:30.0005	NULL	NULL	NULL
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	true	NULL	1969-12-31 16:00:09.986	NULL	2000-12-18 08:42:30.0005	NULL	NULL	NULL
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	true	NULL	1969-12-31 16:00:11.031	NULL	2000-12-18 08:42:30.0005	NULL	NULL	NULL
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	true	NULL	1969-12-31 16:00:11.465	NULL	2000-12-18 08:42:30.0005	NULL	NULL	NULL
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	true	NULL	1969-12-31 16:00:13.589	NULL	2000-12-18 08:42:30.0005	NULL	NULL	NULL
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	true	NULL	2024-11-11 16:42:41.101	NULL	2000-12-18 08:42:30.0005	NULL	NULL	NULL
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	true	NULL	NULL	NULL	2000-12-18 08:42:30.0005	NULL	NULL	NULL
 PREHOOK: query: EXPLAIN VECTORIZATION EXPRESSION  SELECT
   to_unix_timestamp(stimestamp1) AS c1,
   year(stimestamp1),
@@ -291,7 +469,7 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: alltypesorc_string
-                  Statistics: Num rows: 40 Data size: 84 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 52 Data size: 1017 Basic stats: COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
                   Select Operator
@@ -300,9 +478,9 @@ STAGE PLANS:
                     Select Vectorization:
                         className: VectorSelectOperator
                         native: true
-                        projectedOutputColumnNums: [3, 4, 5, 6, 7, 8, 9, 10, 11]
-                        selectExpressions: VectorUDFUnixTimeStampString(col 1:string) -> 3:bigint, VectorUDFYearString(col 1:string, fieldStart 0, fieldLength 4) -> 4:int, VectorUDFMonthString(col 1:string, fieldStart 5, fieldLength 2) -> 5:int, VectorUDFDayOfMonthString(col 1:string, fieldStart 8, fieldLength 2) -> 6:int, VectorUDFDayOfMonthString(col 1:string, fieldStart 8, fieldLength 2) -> 7:int, VectorUDFWeekOfYearString(col 1:string) -> 8:int, VectorUDFHourString(col 1:string, fieldStart 11, fieldLength 2) -> 9:int, VectorUDFMinuteString(col 1:string, fieldStart 14, fieldLength 2) -> 10:int, VectorUDFSecondString(col 1:string, fieldStart 17, fieldLength 2) -> 11:int
-                    Statistics: Num rows: 40 Data size: 1600 Basic stats: COMPLETE Column stats: COMPLETE
+                        projectedOutputColumnNums: [5, 6, 7, 8, 9, 10, 11, 12, 13]
+                        selectExpressions: VectorUDFUnixTimeStampString(col 2:string) -> 5:bigint, VectorUDFYearString(col 2:string, fieldStart 0, fieldLength 4) -> 6:int, VectorUDFMonthString(col 2:string, fieldStart 5, fieldLength 2) -> 7:int, VectorUDFDayOfMonthString(col 2:string, fieldStart 8, fieldLength 2) -> 8:int, VectorUDFDayOfMonthString(col 2:string, fieldStart 8, fieldLength 2) -> 9:int, VectorUDFWeekOfYearString(col 2:string) -> 10:int, VectorUDFHourString(col 2:string, fieldStart 11, fieldLength 2) -> 11:int, VectorUDFMinuteString(col 2:string, fieldStart 14, fieldLength 2) -> 12:int, VectorUDFSecondString(col 2:string, fieldStart 17, fieldLength 2) -> 13:int
+                    Statistics: Num rows: 52 Data size: 2080 Basic stats: COMPLETE Column stats: COMPLETE
                     Reduce Output Operator
                       key expressions: _col0 (type: bigint)
                       sort order: +
@@ -310,7 +488,7 @@ STAGE PLANS:
                           className: VectorReduceSinkObjectHashOperator
                           native: true
                           nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true
-                      Statistics: Num rows: 40 Data size: 1600 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 52 Data size: 2080 Basic stats: COMPLETE Column stats: COMPLETE
                       value expressions: _col1 (type: int), _col2 (type: int), _col3 (type: int), _col4 (type: int), _col5 (type: int), _col6 (type: int), _col7 (type: int), _col8 (type: int)
             Execution mode: vectorized, llap
             LLAP IO: all inputs
@@ -339,13 +517,13 @@ STAGE PLANS:
                     className: VectorSelectOperator
                     native: true
                     projectedOutputColumnNums: [0, 1, 2, 3, 4, 5, 6, 7, 8]
-                Statistics: Num rows: 40 Data size: 1600 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 52 Data size: 2080 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
                   File Sink Vectorization:
                       className: VectorFileSinkOperator
                       native: false
-                  Statistics: Num rows: 40 Data size: 1600 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 52 Data size: 2080 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -387,6 +565,18 @@ ORDER BY c1
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@alltypesorc_string
 #### A masked pattern was here ####
+-2736243926	1883	4	17	17	16	4	14	34
+-62018170411	4	9	22	22	39	18	26	29
+1365579826	2013	4	10	10	15	0	43	46
+206731024925	8521	1	16	16	3	20	42	5
+271201265	1978	8	5	5	31	14	41	5
+501208674	1985	11	18	18	47	16	37	54
+501208674	1985	11	18	18	47	16	37	54
+94573848655	4966	12	4	4	49	9	30	55
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
 NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
 NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
 NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
@@ -473,7 +663,7 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: alltypesorc_string
-                  Statistics: Num rows: 40 Data size: 1684 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 52 Data size: 3097 Basic stats: COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
                   Select Operator
@@ -482,9 +672,9 @@ STAGE PLANS:
                     Select Vectorization:
                         className: VectorSelectOperator
                         native: true
-                        projectedOutputColumnNums: [5, 6, 7, 8, 9, 10, 11, 12, 13]
-                        selectExpressions: LongColEqualLongColumn(col 3:bigint, col 4:bigint)(children: VectorUDFUnixTimeStampTimestamp(col 0:timestamp) -> 3:bigint, VectorUDFUnixTimeStampString(col 1:string) -> 4:bigint) -> 5:boolean, LongColEqualLongColumn(col 3:int, col 4:int)(children: VectorUDFYearTimestamp(col 0:timestamp, field YEAR) -> 3:int, VectorUDFYearString(col 1:string, fieldStart 0, fieldLength 4) -> 4:int) -> 6:boolean, LongColEqualLongColumn(col 3:int, col 4:int)(children: VectorUDFMonthTimestamp(col 0:timestamp, field MONTH) -> 3:int, VectorUDFMonthString(col 1:string, fieldStart 5, fieldLength 2) -> 4:int) -> 7:boolean, LongColEqualLongColumn(col 3:int, col 4:int)(children: VectorUDFDayOfMonthTimestamp(col 0:timestamp, field DAY_OF_MONTH) -> 3:int, VectorUDFDayOfMonthString(col 1:string, fieldStart 8, fieldLength 2) -> 4:int) -> 8:boolean, LongColEqualLongColumn(col 3:int, col 4:int)(children: VectorUDFDayOfMonthTimestamp(col 0:timestamp, field DAY_OF_MONTH) -> 3:
 int, VectorUDFDayOfMonthString(col 1:string, fieldStart 8, fieldLength 2) -> 4:int) -> 9:boolean, LongColEqualLongColumn(col 3:int, col 4:int)(children: VectorUDFWeekOfYearTimestamp(col 0:timestamp, field WEEK_OF_YEAR) -> 3:int, VectorUDFWeekOfYearString(col 1:string) -> 4:int) -> 10:boolean, LongColEqualLongColumn(col 3:int, col 4:int)(children: VectorUDFHourTimestamp(col 0:timestamp, field HOUR_OF_DAY) -> 3:int, VectorUDFHourString(col 1:string, fieldStart 11, fieldLength 2) -> 4:int) -> 11:boolean, LongColEqualLongColumn(col 3:int, col 4:int)(children: VectorUDFMinuteTimestamp(col 0:timestamp, field MINUTE) -> 3:int, VectorUDFMinuteString(col 1:string, fieldStart 14, fieldLength 2) -> 4:int) -> 12:boolean, LongColEqualLongColumn(col 3:int, col 4:int)(children: VectorUDFSecondTimestamp(col 0:timestamp, field SECOND) -> 3:int, VectorUDFSecondString(col 1:string, fieldStart 17, fieldLength 2) -> 4:int) -> 13:boolean
-                    Statistics: Num rows: 40 Data size: 1440 Basic stats: COMPLETE Column stats: COMPLETE
+                        projectedOutputColumnNums: [7, 8, 9, 10, 11, 12, 13, 14, 15]
+                        selectExpressions: LongColEqualLongColumn(col 5:bigint, col 6:bigint)(children: VectorUDFUnixTimeStampTimestamp(col 1:timestamp) -> 5:bigint, VectorUDFUnixTimeStampString(col 2:string) -> 6:bigint) -> 7:boolean, LongColEqualLongColumn(col 5:int, col 6:int)(children: VectorUDFYearTimestamp(col 1:timestamp, field YEAR) -> 5:int, VectorUDFYearString(col 2:string, fieldStart 0, fieldLength 4) -> 6:int) -> 8:boolean, LongColEqualLongColumn(col 5:int, col 6:int)(children: VectorUDFMonthTimestamp(col 1:timestamp, field MONTH) -> 5:int, VectorUDFMonthString(col 2:string, fieldStart 5, fieldLength 2) -> 6:int) -> 9:boolean, LongColEqualLongColumn(col 5:int, col 6:int)(children: VectorUDFDayOfMonthTimestamp(col 1:timestamp, field DAY_OF_MONTH) -> 5:int, VectorUDFDayOfMonthString(col 2:string, fieldStart 8, fieldLength 2) -> 6:int) -> 10:boolean, LongColEqualLongColumn(col 5:int, col 6:int)(children: VectorUDFDayOfMonthTimestamp(col 1:timestamp, field DAY_OF_MONTH) -> 5
 :int, VectorUDFDayOfMonthString(col 2:string, fieldStart 8, fieldLength 2) -> 6:int) -> 11:boolean, LongColEqualLongColumn(col 5:int, col 6:int)(children: VectorUDFWeekOfYearTimestamp(col 1:timestamp, field WEEK_OF_YEAR) -> 5:int, VectorUDFWeekOfYearString(col 2:string) -> 6:int) -> 12:boolean, LongColEqualLongColumn(col 5:int, col 6:int)(children: VectorUDFHourTimestamp(col 1:timestamp, field HOUR_OF_DAY) -> 5:int, VectorUDFHourString(col 2:string, fieldStart 11, fieldLength 2) -> 6:int) -> 13:boolean, LongColEqualLongColumn(col 5:int, col 6:int)(children: VectorUDFMinuteTimestamp(col 1:timestamp, field MINUTE) -> 5:int, VectorUDFMinuteString(col 2:string, fieldStart 14, fieldLength 2) -> 6:int) -> 14:boolean, LongColEqualLongColumn(col 5:int, col 6:int)(children: VectorUDFSecondTimestamp(col 1:timestamp, field SECOND) -> 5:int, VectorUDFSecondString(col 2:string, fieldStart 17, fieldLength 2) -> 6:int) -> 15:boolean
+                    Statistics: Num rows: 52 Data size: 1872 Basic stats: COMPLETE Column stats: COMPLETE
                     Reduce Output Operator
                       key expressions: _col0 (type: boolean)
                       sort order: +
@@ -492,7 +682,7 @@ STAGE PLANS:
                           className: VectorReduceSinkObjectHashOperator
                           native: true
                           nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true
-                      Statistics: Num rows: 40 Data size: 1440 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 52 Data size: 1872 Basic stats: COMPLETE Column stats: COMPLETE
                       value expressions: _col1 (type: boolean), _col2 (type: boolean), _col3 (type: boolean), _col4 (type: boolean), _col5 (type: boolean), _col6 (type: boolean), _col7 (type: boolean), _col8 (type: boolean)
             Execution mode: vectorized, llap
             LLAP IO: all inputs
@@ -521,13 +711,13 @@ STAGE PLANS:
                     className: VectorSelectOperator
                     native: true
                     projectedOutputColumnNums: [0, 1, 2, 3, 4, 5, 6, 7, 8]
-                Statistics: Num rows: 40 Data size: 1440 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 52 Data size: 1872 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
                   File Sink Vectorization:
                       className: VectorFileSinkOperator
                       native: false
-                  Statistics: Num rows: 40 Data size: 1440 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 52 Data size: 1872 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -609,6 +799,18 @@ NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
 NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
 NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
 NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
+false	false	false	false	false	false	false	false	false
+false	false	false	false	false	false	false	false	false
+false	false	false	false	false	false	false	false	false
+false	false	false	false	false	false	false	false	false
 PREHOOK: query: EXPLAIN VECTORIZATION EXPRESSION  SELECT
   to_unix_timestamp(stimestamp1) AS c1,
   year(stimestamp1),
@@ -788,7 +990,7 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: alltypesorc_string
-                  Statistics: Num rows: 40 Data size: 1600 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 52 Data size: 2080 Basic stats: COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
                   Select Operator
@@ -797,12 +999,12 @@ STAGE PLANS:
                     Select Vectorization:
                         className: VectorSelectOperator
                         native: true
-                        projectedOutputColumnNums: [0]
-                    Statistics: Num rows: 40 Data size: 1600 Basic stats: COMPLETE Column stats: COMPLETE
+                        projectedOutputColumnNums: [1]
+                    Statistics: Num rows: 52 Data size: 2080 Basic stats: COMPLETE Column stats: COMPLETE
                     Group By Operator
                       aggregations: min(ctimestamp1), max(ctimestamp1), count(ctimestamp1), count()
                       Group By Vectorization:
-                          aggregators: VectorUDAFMinTimestamp(col 0:timestamp) -> timestamp, VectorUDAFMaxTimestamp(col 0:timestamp) -> timestamp, VectorUDAFCount(col 0:timestamp) -> bigint, VectorUDAFCountStar(*) -> bigint
+                          aggregators: VectorUDAFMinTimestamp(col 1:timestamp) -> timestamp, VectorUDAFMaxTimestamp(col 1:timestamp) -> timestamp, VectorUDAFCount(col 1:timestamp) -> bigint, VectorUDAFCountStar(*) -> bigint
                           className: VectorGroupByOperator
                           groupByMode: HASH
                           native: false
@@ -886,7 +1088,7 @@ FROM alltypesorc_string
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@alltypesorc_string
 #### A masked pattern was here ####
-NULL	NULL	0	40
+0528-10-27 08:15:18.941718273	7160-12-02 06:00:24.81200852	8	52
 PREHOOK: query: EXPLAIN VECTORIZATION EXPRESSION  SELECT
   round(sum(ctimestamp1), 3)
 FROM alltypesorc_string
@@ -915,7 +1117,7 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: alltypesorc_string
-                  Statistics: Num rows: 40 Data size: 1600 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 52 Data size: 2080 Basic stats: COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
                   Select Operator
@@ -924,12 +1126,12 @@ STAGE PLANS:
                     Select Vectorization:
                         className: VectorSelectOperator
                         native: true
-                        projectedOutputColumnNums: [0]
-                    Statistics: Num rows: 40 Data size: 1600 Basic stats: COMPLETE Column stats: COMPLETE
+                        projectedOutputColumnNums: [1]
+                    Statistics: Num rows: 52 Data size: 2080 Basic stats: COMPLETE Column stats: COMPLETE
                     Group By Operator
                       aggregations: sum(ctimestamp1)
                       Group By Vectorization:
-                          aggregators: VectorUDAFSumTimestamp(col 0:timestamp) -> double
+                          aggregators: VectorUDAFSumTimestamp(col 1:timestamp) -> double
                           className: VectorGroupByOperator
                           groupByMode: HASH
                           native: false
@@ -1016,7 +1218,7 @@ FROM alltypesorc_string
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@alltypesorc_string
 #### A masked pattern was here ####
-NULL
+2.89160863229166E11
 PREHOOK: query: EXPLAIN VECTORIZATION EXPRESSION  SELECT
   round(avg(ctimestamp1), 0),
   variance(ctimestamp1) between 8.97077295279421E19 and 8.97077295279422E19,
@@ -1059,7 +1261,7 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: alltypesorc_string
-                  Statistics: Num rows: 40 Data size: 1600 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 52 Data size: 2080 Basic stats: COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
                   Select Operator
@@ -1068,13 +1270,13 @@ STAGE PLANS:
                     Select Vectorization:
                         className: VectorSelectOperator
                         native: true
-                        projectedOutputColumnNums: [0, 3, 6]
-                        selectExpressions: CastTimestampToDouble(col 0:timestamp) -> 3:double, DoubleColMultiplyDoubleColumn(col 4:double, col 5:double)(children: CastTimestampToDouble(col 0:timestamp) -> 4:double, CastTimestampToDouble(col 0:timestamp) -> 5:double) -> 6:double
-                    Statistics: Num rows: 40 Data size: 1600 Basic stats: COMPLETE Column stats: COMPLETE
+                        projectedOutputColumnNums: [1, 5, 8]
+                        selectExpressions: CastTimestampToDouble(col 1:timestamp) -> 5:double, DoubleColMultiplyDoubleColumn(col 6:double, col 7:double)(children: CastTimestampToDouble(col 1:timestamp) -> 6:double, CastTimestampToDouble(col 1:timestamp) -> 7:double) -> 8:double
+                    Statistics: Num rows: 52 Data size: 2080 Basic stats: COMPLETE Column stats: COMPLETE
                     Group By Operator
                       aggregations: sum(_col0), count(_col0), sum(_col2), sum(_col1)
                       Group By Vectorization:
-                          aggregators: VectorUDAFSumTimestamp(col 0:timestamp) -> double, VectorUDAFCount(col 0:timestamp) -> bigint, VectorUDAFSumDouble(col 6:double) -> double, VectorUDAFSumDouble(col 3:double) -> double
+                          aggregators: VectorUDAFSumTimestamp(col 1:timestamp) -> double, VectorUDAFCount(col 1:timestamp) -> bigint, VectorUDAFSumDouble(col 8:double) -> double, VectorUDAFSumDouble(col 5:double) -> double
                           className: VectorGroupByOperator
                           groupByMode: HASH
                           native: false
@@ -1175,4 +1377,4 @@ FROM alltypesorc_string
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@alltypesorc_string
 #### A masked pattern was here ####
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
+3.6145107904E10	false	false	false	7.5245155692476E10	7.5245155692476E10	7.5245155692476E10	8.0440455033059E10


[45/50] [abbrv] hive git commit: HIVE-19250: Schema column definition inconsistencies in MySQL (Naveen Gangam, reviewed by Aihua Xu)

Posted by vg...@apache.org.
HIVE-19250: Schema column definition inconsistencies in MySQL (Naveen Gangam, reviewed by Aihua Xu)


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

Branch: refs/heads/branch-3.0.0
Commit: efe9ab8ed6456f7a3e53c643974d4b7822ea3707
Parents: 3ea0356
Author: Naveen Gangam <ng...@apache.org>
Authored: Mon Apr 23 13:51:45 2018 -0400
Committer: Naveen Gangam <ng...@apache.org>
Committed: Tue May 15 17:14:58 2018 -0400

----------------------------------------------------------------------
 metastore/scripts/upgrade/mysql/039-HIVE-12274.mysql.sql | 10 +++++-----
 1 file changed, 5 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/efe9ab8e/metastore/scripts/upgrade/mysql/039-HIVE-12274.mysql.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/mysql/039-HIVE-12274.mysql.sql b/metastore/scripts/upgrade/mysql/039-HIVE-12274.mysql.sql
index cdaf286..bd6bd29 100644
--- a/metastore/scripts/upgrade/mysql/039-HIVE-12274.mysql.sql
+++ b/metastore/scripts/upgrade/mysql/039-HIVE-12274.mysql.sql
@@ -1,14 +1,14 @@
 ALTER TABLE COLUMNS_V2 MODIFY TYPE_NAME MEDIUMTEXT;
-ALTER TABLE TABLE_PARAMS MODIFY PARAM_VALUE MEDIUMTEXT;
-ALTER TABLE SERDE_PARAMS MODIFY PARAM_VALUE MEDIUMTEXT;
-ALTER TABLE SD_PARAMS MODIFY PARAM_VALUE MEDIUMTEXT;
+ALTER TABLE TABLE_PARAMS MODIFY PARAM_VALUE MEDIUMTEXT CHARACTER SET latin1 COLLATE latin1_bin;
+ALTER TABLE SERDE_PARAMS MODIFY PARAM_VALUE MEDIUMTEXT CHARACTER SET latin1 COLLATE latin1_bin;
+ALTER TABLE SD_PARAMS MODIFY PARAM_VALUE MEDIUMTEXT CHARACTER SET latin1 COLLATE latin1_bin;
 
 ALTER TABLE TBLS MODIFY TBL_NAME varchar(256) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL;
-ALTER TABLE NOTIFICATION_LOG MODIFY TBL_NAME varchar(256) CHARACTER SET latin1 COLLATE latin1_bin;
+ALTER TABLE NOTIFICATION_LOG MODIFY TBL_NAME varchar(256);
 ALTER TABLE PARTITION_EVENTS MODIFY TBL_NAME varchar(256) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL;
 ALTER TABLE TAB_COL_STATS MODIFY TABLE_NAME varchar(256) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL;
 ALTER TABLE PART_COL_STATS MODIFY TABLE_NAME varchar(256) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL;
-ALTER TABLE COMPLETED_TXN_COMPONENTS MODIFY CTC_TABLE varchar(256) CHARACTER SET latin1 COLLATE latin1_bin;
+ALTER TABLE COMPLETED_TXN_COMPONENTS MODIFY CTC_TABLE varchar(256);
 
 ALTER TABLE COLUMNS_V2 MODIFY COLUMN_NAME varchar(767) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL;
 ALTER TABLE PART_COL_PRIVS MODIFY COLUMN_NAME varchar(767) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL;


[08/50] [abbrv] hive git commit: HIVE-19298 : Fix operator tree of CTAS for Druid Storage Handler (Slim Bouguerra via Ashutosh Chauhan)

Posted by vg...@apache.org.
HIVE-19298 : Fix operator tree of CTAS for Druid Storage Handler (Slim Bouguerra via Ashutosh Chauhan)

Signed-off-by: Ashutosh Chauhan <ha...@apache.org>


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

Branch: refs/heads/branch-3.0.0
Commit: b135724029272963f16b57830c630ce9a69fe8b5
Parents: 71d77ab
Author: Slim Bouguerra <sl...@gmail.com>
Authored: Mon May 7 22:56:19 2018 -0700
Committer: Vineet Garg <vg...@apache.org>
Committed: Tue May 8 16:05:12 2018 -0700

----------------------------------------------------------------------
 .../results/positive/accumulo_queries.q.out     |  22 ++--
 .../accumulo_single_sourced_multi_insert.q.out  |  35 +++---
 .../src/test/results/positive/hbase_ddl.q.out   |   9 +-
 .../test/results/positive/hbase_queries.q.out   |  22 ++--
 .../hbase_single_sourced_multi_insert.q.out     |  35 +++---
 .../src/test/results/positive/hbasestats.q.out  |   9 +-
 .../org/apache/hadoop/hive/ql/exec/DDLTask.java |  62 ++++++++---
 .../org/apache/hadoop/hive/ql/exec/Task.java    |   1 -
 .../ql/optimizer/QueryPlanPostProcessor.java    |   6 +-
 .../hadoop/hive/ql/parse/GenTezProcContext.java |  20 ++--
 .../hadoop/hive/ql/parse/ParseContext.java      |  23 ++--
 .../hadoop/hive/ql/parse/SemanticAnalyzer.java  | 106 +++++++++++--------
 .../hadoop/hive/ql/parse/TaskCompiler.java      |  30 +++---
 .../org/apache/hadoop/hive/ql/plan/DDLWork.java |  21 ++--
 .../hive/ql/plan/InsertCommitHookDesc.java      |  41 +++++++
 .../hadoop/hive/ql/plan/InsertTableDesc.java    |  40 -------
 .../druid/druidmini_dynamic_partition.q.out     |  28 ++---
 .../clientpositive/druid/druidmini_mv.q.out     |  18 ++--
 .../hive/metastore/DefaultHiveMetaHook.java     |   2 +-
 19 files changed, 269 insertions(+), 261 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/b1357240/accumulo-handler/src/test/results/positive/accumulo_queries.q.out
----------------------------------------------------------------------
diff --git a/accumulo-handler/src/test/results/positive/accumulo_queries.q.out b/accumulo-handler/src/test/results/positive/accumulo_queries.q.out
index 78a2a8a..f7e7699 100644
--- a/accumulo-handler/src/test/results/positive/accumulo_queries.q.out
+++ b/accumulo-handler/src/test/results/positive/accumulo_queries.q.out
@@ -40,9 +40,8 @@ POSTHOOK: query: EXPLAIN FROM src INSERT OVERWRITE TABLE accumulo_table_1 SELECT
 POSTHOOK: type: QUERY
 STAGE DEPENDENCIES:
   Stage-0 is a root stage
-  Stage-2
   Stage-1 is a root stage
-  Stage-3 is a root stage
+  Stage-2 is a root stage
 
 STAGE PLANS:
   Stage: Stage-0
@@ -53,15 +52,11 @@ STAGE PLANS:
           properties:
             COLUMN_STATS_ACCURATE 
 
-  Stage: Stage-2
-      Insert operator:
-        Insert
-
   Stage: Stage-1
       Pre Insert operator:
         Pre-Insert task
 
-  Stage: Stage-3
+  Stage: Stage-2
     Map Reduce
       Map Operator Tree:
           TableScan
@@ -506,10 +501,9 @@ ON (x.key = Y.key)
 POSTHOOK: type: QUERY
 STAGE DEPENDENCIES:
   Stage-0 is a root stage
-  Stage-2
   Stage-1 is a root stage
-  Stage-4 is a root stage
-  Stage-3 depends on stages: Stage-4
+  Stage-3 is a root stage
+  Stage-2 depends on stages: Stage-3
 
 STAGE PLANS:
   Stage: Stage-0
@@ -520,15 +514,11 @@ STAGE PLANS:
           properties:
             COLUMN_STATS_ACCURATE 
 
-  Stage: Stage-2
-      Insert operator:
-        Insert
-
   Stage: Stage-1
       Pre Insert operator:
         Pre-Insert task
 
-  Stage: Stage-4
+  Stage: Stage-3
     Map Reduce
       Map Operator Tree:
           TableScan
@@ -564,7 +554,7 @@ STAGE PLANS:
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                 serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
 
-  Stage: Stage-3
+  Stage: Stage-2
     Map Reduce
       Map Operator Tree:
           TableScan

http://git-wip-us.apache.org/repos/asf/hive/blob/b1357240/accumulo-handler/src/test/results/positive/accumulo_single_sourced_multi_insert.q.out
----------------------------------------------------------------------
diff --git a/accumulo-handler/src/test/results/positive/accumulo_single_sourced_multi_insert.q.out b/accumulo-handler/src/test/results/positive/accumulo_single_sourced_multi_insert.q.out
index 0efe121..5905ecd 100644
--- a/accumulo-handler/src/test/results/positive/accumulo_single_sourced_multi_insert.q.out
+++ b/accumulo-handler/src/test/results/positive/accumulo_single_sourced_multi_insert.q.out
@@ -34,16 +34,15 @@ select value,"" where a.key > 50 AND a.key < 100
 POSTHOOK: type: QUERY
 STAGE DEPENDENCIES:
   Stage-0 is a root stage
-  Stage-2
-  Stage-5 depends on stages: Stage-3, Stage-2, Stage-1
+  Stage-4 depends on stages: Stage-2, Stage-0, Stage-1
   Stage-1 is a root stage
-  Stage-4 is a root stage
-  Stage-10 depends on stages: Stage-4 , consists of Stage-7, Stage-6, Stage-8
-  Stage-7
-  Stage-3 depends on stages: Stage-7, Stage-6, Stage-9
+  Stage-3 is a root stage
+  Stage-9 depends on stages: Stage-3 , consists of Stage-6, Stage-5, Stage-7
   Stage-6
-  Stage-8
-  Stage-9 depends on stages: Stage-8
+  Stage-2 depends on stages: Stage-6, Stage-5, Stage-8
+  Stage-5
+  Stage-7
+  Stage-8 depends on stages: Stage-7
 
 STAGE PLANS:
   Stage: Stage-0
@@ -54,11 +53,7 @@ STAGE PLANS:
           properties:
             COLUMN_STATS_ACCURATE 
 
-  Stage: Stage-2
-      Insert operator:
-        Insert
-
-  Stage: Stage-5
+  Stage: Stage-4
     Stats Work
       Basic Stats Work:
       Column Stats Desc:
@@ -70,7 +65,7 @@ STAGE PLANS:
       Pre Insert operator:
         Pre-Insert task
 
-  Stage: Stage-4
+  Stage: Stage-3
     Map Reduce
       Map Operator Tree:
           TableScan
@@ -133,16 +128,16 @@ STAGE PLANS:
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                 serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
 
-  Stage: Stage-10
+  Stage: Stage-9
     Conditional Operator
 
-  Stage: Stage-7
+  Stage: Stage-6
     Move Operator
       files:
           hdfs directory: true
 #### A masked pattern was here ####
 
-  Stage: Stage-3
+  Stage: Stage-2
     Move Operator
       tables:
           replace: true
@@ -152,7 +147,7 @@ STAGE PLANS:
               serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
               name: default.src_x1
 
-  Stage: Stage-6
+  Stage: Stage-5
     Map Reduce
       Map Operator Tree:
           TableScan
@@ -164,7 +159,7 @@ STAGE PLANS:
                   serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
                   name: default.src_x1
 
-  Stage: Stage-8
+  Stage: Stage-7
     Map Reduce
       Map Operator Tree:
           TableScan
@@ -176,7 +171,7 @@ STAGE PLANS:
                   serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
                   name: default.src_x1
 
-  Stage: Stage-9
+  Stage: Stage-8
     Move Operator
       files:
           hdfs directory: true

http://git-wip-us.apache.org/repos/asf/hive/blob/b1357240/hbase-handler/src/test/results/positive/hbase_ddl.q.out
----------------------------------------------------------------------
diff --git a/hbase-handler/src/test/results/positive/hbase_ddl.q.out b/hbase-handler/src/test/results/positive/hbase_ddl.q.out
index bf7da98..9d16034 100644
--- a/hbase-handler/src/test/results/positive/hbase_ddl.q.out
+++ b/hbase-handler/src/test/results/positive/hbase_ddl.q.out
@@ -40,9 +40,8 @@ POSTHOOK: query: EXPLAIN FROM src INSERT OVERWRITE TABLE hbase_table_1 SELECT *
 POSTHOOK: type: QUERY
 STAGE DEPENDENCIES:
   Stage-0 is a root stage
-  Stage-2
   Stage-1 is a root stage
-  Stage-3 is a root stage
+  Stage-2 is a root stage
 
 STAGE PLANS:
   Stage: Stage-0
@@ -53,15 +52,11 @@ STAGE PLANS:
           properties:
             COLUMN_STATS_ACCURATE 
 
-  Stage: Stage-2
-      Insert operator:
-        Insert
-
   Stage: Stage-1
       Pre Insert operator:
         Pre-Insert task
 
-  Stage: Stage-3
+  Stage: Stage-2
     Map Reduce
       Map Operator Tree:
           TableScan

http://git-wip-us.apache.org/repos/asf/hive/blob/b1357240/hbase-handler/src/test/results/positive/hbase_queries.q.out
----------------------------------------------------------------------
diff --git a/hbase-handler/src/test/results/positive/hbase_queries.q.out b/hbase-handler/src/test/results/positive/hbase_queries.q.out
index 8a91b66..eb91bf9 100644
--- a/hbase-handler/src/test/results/positive/hbase_queries.q.out
+++ b/hbase-handler/src/test/results/positive/hbase_queries.q.out
@@ -40,9 +40,8 @@ POSTHOOK: query: EXPLAIN FROM src INSERT OVERWRITE TABLE hbase_table_1 SELECT *
 POSTHOOK: type: QUERY
 STAGE DEPENDENCIES:
   Stage-0 is a root stage
-  Stage-2
   Stage-1 is a root stage
-  Stage-3 is a root stage
+  Stage-2 is a root stage
 
 STAGE PLANS:
   Stage: Stage-0
@@ -53,15 +52,11 @@ STAGE PLANS:
           properties:
             COLUMN_STATS_ACCURATE 
 
-  Stage: Stage-2
-      Insert operator:
-        Insert
-
   Stage: Stage-1
       Pre Insert operator:
         Pre-Insert task
 
-  Stage: Stage-3
+  Stage: Stage-2
     Map Reduce
       Map Operator Tree:
           TableScan
@@ -506,10 +501,9 @@ ON (x.key = Y.key)
 POSTHOOK: type: QUERY
 STAGE DEPENDENCIES:
   Stage-0 is a root stage
-  Stage-2
   Stage-1 is a root stage
-  Stage-4 is a root stage
-  Stage-3 depends on stages: Stage-4
+  Stage-3 is a root stage
+  Stage-2 depends on stages: Stage-3
 
 STAGE PLANS:
   Stage: Stage-0
@@ -520,15 +514,11 @@ STAGE PLANS:
           properties:
             COLUMN_STATS_ACCURATE 
 
-  Stage: Stage-2
-      Insert operator:
-        Insert
-
   Stage: Stage-1
       Pre Insert operator:
         Pre-Insert task
 
-  Stage: Stage-4
+  Stage: Stage-3
     Map Reduce
       Map Operator Tree:
           TableScan
@@ -564,7 +554,7 @@ STAGE PLANS:
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                 serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
 
-  Stage: Stage-3
+  Stage: Stage-2
     Map Reduce
       Map Operator Tree:
           TableScan

http://git-wip-us.apache.org/repos/asf/hive/blob/b1357240/hbase-handler/src/test/results/positive/hbase_single_sourced_multi_insert.q.out
----------------------------------------------------------------------
diff --git a/hbase-handler/src/test/results/positive/hbase_single_sourced_multi_insert.q.out b/hbase-handler/src/test/results/positive/hbase_single_sourced_multi_insert.q.out
index 7e47a65..86a9fea 100644
--- a/hbase-handler/src/test/results/positive/hbase_single_sourced_multi_insert.q.out
+++ b/hbase-handler/src/test/results/positive/hbase_single_sourced_multi_insert.q.out
@@ -34,16 +34,15 @@ select value,"" where a.key > 50 AND a.key < 100
 POSTHOOK: type: QUERY
 STAGE DEPENDENCIES:
   Stage-0 is a root stage
-  Stage-2
-  Stage-5 depends on stages: Stage-3, Stage-2, Stage-1
+  Stage-4 depends on stages: Stage-2, Stage-0, Stage-1
   Stage-1 is a root stage
-  Stage-4 is a root stage
-  Stage-10 depends on stages: Stage-4 , consists of Stage-7, Stage-6, Stage-8
-  Stage-7
-  Stage-3 depends on stages: Stage-7, Stage-6, Stage-9
+  Stage-3 is a root stage
+  Stage-9 depends on stages: Stage-3 , consists of Stage-6, Stage-5, Stage-7
   Stage-6
-  Stage-8
-  Stage-9 depends on stages: Stage-8
+  Stage-2 depends on stages: Stage-6, Stage-5, Stage-8
+  Stage-5
+  Stage-7
+  Stage-8 depends on stages: Stage-7
 
 STAGE PLANS:
   Stage: Stage-0
@@ -54,11 +53,7 @@ STAGE PLANS:
           properties:
             COLUMN_STATS_ACCURATE 
 
-  Stage: Stage-2
-      Insert operator:
-        Insert
-
-  Stage: Stage-5
+  Stage: Stage-4
     Stats Work
       Basic Stats Work:
       Column Stats Desc:
@@ -70,7 +65,7 @@ STAGE PLANS:
       Pre Insert operator:
         Pre-Insert task
 
-  Stage: Stage-4
+  Stage: Stage-3
     Map Reduce
       Map Operator Tree:
           TableScan
@@ -133,16 +128,16 @@ STAGE PLANS:
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                 serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
 
-  Stage: Stage-10
+  Stage: Stage-9
     Conditional Operator
 
-  Stage: Stage-7
+  Stage: Stage-6
     Move Operator
       files:
           hdfs directory: true
 #### A masked pattern was here ####
 
-  Stage: Stage-3
+  Stage: Stage-2
     Move Operator
       tables:
           replace: true
@@ -152,7 +147,7 @@ STAGE PLANS:
               serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
               name: default.src_x1
 
-  Stage: Stage-6
+  Stage: Stage-5
     Map Reduce
       Map Operator Tree:
           TableScan
@@ -164,7 +159,7 @@ STAGE PLANS:
                   serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
                   name: default.src_x1
 
-  Stage: Stage-8
+  Stage: Stage-7
     Map Reduce
       Map Operator Tree:
           TableScan
@@ -176,7 +171,7 @@ STAGE PLANS:
                   serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
                   name: default.src_x1
 
-  Stage: Stage-9
+  Stage: Stage-8
     Move Operator
       files:
           hdfs directory: true

http://git-wip-us.apache.org/repos/asf/hive/blob/b1357240/hbase-handler/src/test/results/positive/hbasestats.q.out
----------------------------------------------------------------------
diff --git a/hbase-handler/src/test/results/positive/hbasestats.q.out b/hbase-handler/src/test/results/positive/hbasestats.q.out
index 92cf8cc..4724ad6 100644
--- a/hbase-handler/src/test/results/positive/hbasestats.q.out
+++ b/hbase-handler/src/test/results/positive/hbasestats.q.out
@@ -63,9 +63,8 @@ POSTHOOK: query: explain INSERT OVERWRITE TABLE users SELECT 'user1', 'IA', 'USA
 POSTHOOK: type: QUERY
 STAGE DEPENDENCIES:
   Stage-0 is a root stage
-  Stage-2
   Stage-1 is a root stage
-  Stage-3 is a root stage
+  Stage-2 is a root stage
 
 STAGE PLANS:
   Stage: Stage-0
@@ -76,15 +75,11 @@ STAGE PLANS:
           properties:
             COLUMN_STATS_ACCURATE 
 
-  Stage: Stage-2
-      Insert operator:
-        Insert
-
   Stage: Stage-1
       Pre Insert operator:
         Pre-Insert task
 
-  Stage: Stage-3
+  Stage: Stage-2
     Map Reduce
       Map Operator Tree:
           TableScan

http://git-wip-us.apache.org/repos/asf/hive/blob/b1357240/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
index ed31348..b3c95eb 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
@@ -53,7 +53,11 @@ import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 import java.util.concurrent.ExecutionException;
 
+import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Lists;
+import com.google.common.util.concurrent.ListenableFuture;
 
 import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.fs.FSDataOutputStream;
@@ -149,7 +153,6 @@ import org.apache.hadoop.hive.ql.lockmgr.HiveLockMode;
 import org.apache.hadoop.hive.ql.lockmgr.HiveLockObject;
 import org.apache.hadoop.hive.ql.lockmgr.HiveLockObject.HiveLockObjectData;
 import org.apache.hadoop.hive.ql.lockmgr.HiveTxnManager;
-import org.apache.hadoop.hive.ql.lockmgr.LockException;
 import org.apache.hadoop.hive.ql.metadata.CheckConstraint;
 import org.apache.hadoop.hive.ql.metadata.CheckResult;
 import org.apache.hadoop.hive.ql.metadata.DefaultConstraint;
@@ -211,7 +214,7 @@ import org.apache.hadoop.hive.ql.plan.DropWMTriggerDesc;
 import org.apache.hadoop.hive.ql.plan.FileMergeDesc;
 import org.apache.hadoop.hive.ql.plan.GrantDesc;
 import org.apache.hadoop.hive.ql.plan.GrantRevokeRoleDDL;
-import org.apache.hadoop.hive.ql.plan.InsertTableDesc;
+import org.apache.hadoop.hive.ql.plan.InsertCommitHookDesc;
 import org.apache.hadoop.hive.ql.plan.KillQueryDesc;
 import org.apache.hadoop.hive.ql.plan.ListBucketingCtx;
 import org.apache.hadoop.hive.ql.plan.LoadMultiFilesDesc;
@@ -291,10 +294,39 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.stringtemplate.v4.ST;
 
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.collect.Iterables;
-import com.google.common.collect.Lists;
-import com.google.common.util.concurrent.ListenableFuture;
+import java.io.BufferedWriter;
+import java.io.DataOutputStream;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.OutputStreamWriter;
+import java.io.Serializable;
+import java.io.Writer;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.nio.charset.StandardCharsets;
+import java.sql.SQLException;
+import java.util.AbstractList;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+import java.util.SortedSet;
+import java.util.TreeMap;
+import java.util.TreeSet;
+import java.util.concurrent.ExecutionException;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import static org.apache.commons.lang.StringUtils.join;
+import static org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.META_TABLE_STORAGE;
 
 /**
  * DDLTask implementation.
@@ -602,9 +634,9 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
       if (cacheMetadataDesc != null) {
         return cacheMetadata(db, cacheMetadataDesc);
       }
-      InsertTableDesc insertTableDesc = work.getInsertTableDesc();
-      if (insertTableDesc != null) {
-        return insertCommitWork(db, insertTableDesc);
+      InsertCommitHookDesc insertCommitHookDesc = work.getInsertCommitHookDesc();
+      if (insertCommitHookDesc != null) {
+        return insertCommitWork(db, insertCommitHookDesc);
       }
       PreInsertTableDesc preInsertTableDesc = work.getPreInsertTableDesc();
       if (preInsertTableDesc != null) {
@@ -860,22 +892,22 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
     return 0;
   }
 
-  private int insertCommitWork(Hive db, InsertTableDesc insertTableDesc) throws MetaException {
+  private int insertCommitWork(Hive db, InsertCommitHookDesc insertCommitHookDesc) throws MetaException {
     boolean failed = true;
-    HiveMetaHook hook = insertTableDesc.getTable().getStorageHandler().getMetaHook();
+    HiveMetaHook hook = insertCommitHookDesc.getTable().getStorageHandler().getMetaHook();
     if (hook == null || !(hook instanceof DefaultHiveMetaHook)) {
       return 0;
     }
     DefaultHiveMetaHook hiveMetaHook = (DefaultHiveMetaHook) hook;
     try {
-      hiveMetaHook.commitInsertTable(insertTableDesc.getTable().getTTable(),
-              insertTableDesc.isOverwrite()
+      hiveMetaHook.commitInsertTable(insertCommitHookDesc.getTable().getTTable(),
+              insertCommitHookDesc.isOverwrite()
       );
       failed = false;
     } finally {
       if (failed) {
-        hiveMetaHook.rollbackInsertTable(insertTableDesc.getTable().getTTable(),
-                insertTableDesc.isOverwrite()
+        hiveMetaHook.rollbackInsertTable(insertCommitHookDesc.getTable().getTTable(),
+                insertCommitHookDesc.isOverwrite()
         );
       }
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/b1357240/ql/src/java/org/apache/hadoop/hive/ql/exec/Task.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/Task.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/Task.java
index 1e8857b..e16411b 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/Task.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/Task.java
@@ -68,7 +68,6 @@ public abstract class Task<T extends Serializable> implements Serializable, Node
   protected static transient Logger LOG = LoggerFactory.getLogger(Task.class);
   protected int taskTag;
   private boolean isLocalMode =false;
-  private boolean retryCmdWhenFail = false;
 
   public static final int NO_TAG = 0;
   public static final int COMMON_JOIN = 1;

http://git-wip-us.apache.org/repos/asf/hive/blob/b1357240/ql/src/java/org/apache/hadoop/hive/ql/optimizer/QueryPlanPostProcessor.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/QueryPlanPostProcessor.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/QueryPlanPostProcessor.java
index c0ce684..a91f45e 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/QueryPlanPostProcessor.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/QueryPlanPostProcessor.java
@@ -17,9 +17,6 @@
  */
 package org.apache.hadoop.hive.ql.optimizer;
 
-import java.util.List;
-import java.util.Set;
-
 import org.apache.hadoop.hive.ql.exec.FileSinkOperator;
 import org.apache.hadoop.hive.ql.exec.Operator;
 import org.apache.hadoop.hive.ql.exec.OperatorUtils;
@@ -53,6 +50,9 @@ import org.apache.hadoop.hive.ql.plan.TezWork;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.util.List;
+import java.util.Set;
+
 /**
  * Finds Acid FileSinkDesc objects which can be created in the physical (disconnected) plan, e.g.
  * {@link org.apache.hadoop.hive.ql.parse.GenTezUtils#removeUnionOperators(GenTezProcContext, BaseWork, int)}

http://git-wip-us.apache.org/repos/asf/hive/blob/b1357240/ql/src/java/org/apache/hadoop/hive/ql/parse/GenTezProcContext.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/GenTezProcContext.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/GenTezProcContext.java
index 0df581a..f977fc1 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/GenTezProcContext.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/GenTezProcContext.java
@@ -18,16 +18,6 @@
 
 package org.apache.hadoop.hive.ql.parse;
 
-import java.io.Serializable;
-import java.util.HashMap;
-import java.util.LinkedHashMap;
-import java.util.LinkedHashSet;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.atomic.AtomicInteger;
-
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.ql.exec.AppMasterEventOperator;
@@ -55,6 +45,16 @@ import org.apache.hadoop.hive.ql.plan.TezEdgeProperty;
 import org.apache.hadoop.hive.ql.plan.TezWork;
 import org.apache.hadoop.hive.ql.plan.UnionWork;
 
+import java.io.Serializable;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.LinkedHashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+
 /**
  * GenTezProcContext. GenTezProcContext maintains information
  * about the tasks and operators as we walk the operator tree

http://git-wip-us.apache.org/repos/asf/hive/blob/b1357240/ql/src/java/org/apache/hadoop/hive/ql/parse/ParseContext.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/ParseContext.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/ParseContext.java
index d890b31..89121e3 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/ParseContext.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/ParseContext.java
@@ -18,16 +18,6 @@
 
 package org.apache.hadoop.hive.ql.parse;
 
-import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.ql.Context;
 import org.apache.hadoop.hive.ql.QueryProperties;
@@ -61,6 +51,16 @@ import org.apache.hadoop.hive.ql.plan.LoadTableDesc;
 import org.apache.hadoop.hive.ql.plan.MapJoinDesc;
 import org.apache.hadoop.hive.ql.plan.TableDesc;
 
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
 /**
  * Parse Context: The current parse context. This is passed to the optimizer
  * which then transforms the operator tree using the parse context. All the
@@ -122,7 +122,7 @@ public class ParseContext {
   private MaterializedViewDesc materializedViewUpdateDesc;
   private boolean reduceSinkAddedBySortedDynPartition;
 
-  private Map<SelectOperator, Table> viewProjectToViewSchema;  
+  private Map<SelectOperator, Table> viewProjectToViewSchema;
   private ColumnAccessInfo columnAccessInfo;
   private boolean needViewColumnAuthorization;
   private Set<FileSinkDesc> acidFileSinks = Collections.emptySet();
@@ -141,7 +141,6 @@ public class ParseContext {
   }
 
   /**
-   * @param conf
    * @param opToPartPruner
    *          map from table scan operator to partition pruner
    * @param opToPartList

http://git-wip-us.apache.org/repos/asf/hive/blob/b1357240/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
index 1ab7158..ff952b6 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
@@ -18,33 +18,12 @@
 
 package org.apache.hadoop.hive.ql.parse;
 
-import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.HIVESTATSDBCLASS;
-
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.io.Serializable;
-import java.security.AccessControlException;
-import java.util.ArrayDeque;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Deque;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.LinkedHashMap;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Queue;
-import java.util.Set;
-import java.util.TreeSet;
-import java.util.UUID;
-import java.util.function.Supplier;
-import java.util.regex.Pattern;
-import java.util.regex.PatternSyntaxException;
-import java.util.stream.Collectors;
-
+import com.google.common.base.Splitter;
+import com.google.common.base.Strings;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Sets;
+import com.google.common.math.IntMath;
+import com.google.common.math.LongMath;
 import org.antlr.runtime.ClassicToken;
 import org.antlr.runtime.CommonToken;
 import org.antlr.runtime.Token;
@@ -65,9 +44,9 @@ import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.hive.common.FileUtils;
 import org.apache.hadoop.hive.common.ObjectPair;
 import org.apache.hadoop.hive.common.StatsSetupConst;
+import org.apache.hadoop.hive.common.StatsSetupConst.StatDB;
 import org.apache.hadoop.hive.common.ValidTxnList;
 import org.apache.hadoop.hive.common.ValidTxnWriteIdList;
-import org.apache.hadoop.hive.common.StatsSetupConst.StatDB;
 import org.apache.hadoop.hive.common.metrics.common.MetricsConstant;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
@@ -121,6 +100,7 @@ import org.apache.hadoop.hive.ql.exec.Task;
 import org.apache.hadoop.hive.ql.exec.TaskFactory;
 import org.apache.hadoop.hive.ql.exec.UnionOperator;
 import org.apache.hadoop.hive.ql.exec.Utilities;
+import org.apache.hadoop.hive.ql.exec.tez.TezTask;
 import org.apache.hadoop.hive.ql.hooks.Entity;
 import org.apache.hadoop.hive.ql.hooks.ReadEntity;
 import org.apache.hadoop.hive.ql.hooks.WriteEntity;
@@ -185,6 +165,7 @@ import org.apache.hadoop.hive.ql.parse.WindowingSpec.WindowType;
 import org.apache.hadoop.hive.ql.plan.AggregationDesc;
 import org.apache.hadoop.hive.ql.plan.AlterTableDesc;
 import org.apache.hadoop.hive.ql.plan.AlterTableDesc.AlterTableTypes;
+import org.apache.hadoop.hive.ql.plan.BaseWork;
 import org.apache.hadoop.hive.ql.plan.CreateTableDesc;
 import org.apache.hadoop.hive.ql.plan.CreateTableLikeDesc;
 import org.apache.hadoop.hive.ql.plan.CreateViewDesc;
@@ -203,7 +184,7 @@ import org.apache.hadoop.hive.ql.plan.FilterDesc.SampleDesc;
 import org.apache.hadoop.hive.ql.plan.ForwardDesc;
 import org.apache.hadoop.hive.ql.plan.GroupByDesc;
 import org.apache.hadoop.hive.ql.plan.HiveOperation;
-import org.apache.hadoop.hive.ql.plan.InsertTableDesc;
+import org.apache.hadoop.hive.ql.plan.InsertCommitHookDesc;
 import org.apache.hadoop.hive.ql.plan.JoinCondDesc;
 import org.apache.hadoop.hive.ql.plan.JoinDesc;
 import org.apache.hadoop.hive.ql.plan.LateralViewForwardDesc;
@@ -230,9 +211,9 @@ import org.apache.hadoop.hive.ql.plan.ptf.PartitionedTableFunctionDef;
 import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject;
 import org.apache.hadoop.hive.ql.session.SessionState;
 import org.apache.hadoop.hive.ql.session.SessionState.ResourceType;
-import org.apache.hadoop.hive.ql.udf.generic.GenericUDFArray;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFEvaluator;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFEvaluator.Mode;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDFArray;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDFCardinalityViolation;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDFHash;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDFMurmurHash;
@@ -270,12 +251,33 @@ import org.apache.hadoop.mapred.OutputFormat;
 import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
 import org.apache.hadoop.security.UserGroupInformation;
 
-import com.google.common.base.Splitter;
-import com.google.common.base.Strings;
-import com.google.common.collect.Iterables;
-import com.google.common.collect.Sets;
-import com.google.common.math.IntMath;
-import com.google.common.math.LongMath;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.Serializable;
+import java.security.AccessControlException;
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Optional;
+import java.util.Queue;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.UUID;
+import java.util.function.Supplier;
+import java.util.regex.Pattern;
+import java.util.regex.PatternSyntaxException;
+import java.util.stream.Collectors;
+
+import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.HIVESTATSDBCLASS;
 
 /**
  * Implementation of the semantic analyzer. It generates the query plan.
@@ -7293,7 +7295,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
         // true if it is insert overwrite.
         boolean overwrite = !qb.getParseInfo().isInsertIntoTable(
             String.format("%s.%s", dest_tab.getDbName(), dest_tab.getTableName()));
-        createInsertDesc(dest_tab, overwrite);
+        createPreInsertDesc(dest_tab, overwrite);
       }
 
       if (dest_tab.isMaterializedView()) {
@@ -7571,7 +7573,6 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
         ltd.setInsertOverwrite(true);
       }
     }
-
     if (SessionState.get().isHiveServerQuery() &&
         null != table_desc &&
         table_desc.getSerdeClassName().equalsIgnoreCase(ThriftJDBCBinarySerDe.class.getName()) &&
@@ -7886,17 +7887,14 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
     return dpCtx;
   }
 
-  private void createInsertDesc(Table table, boolean overwrite) {
-    Task<? extends Serializable>[] tasks = new Task[this.rootTasks.size()];
-    tasks = this.rootTasks.toArray(tasks);
+  private void createPreInsertDesc(Table table, boolean overwrite) {
     PreInsertTableDesc preInsertTableDesc = new PreInsertTableDesc(table, overwrite);
-    InsertTableDesc insertTableDesc = new InsertTableDesc(table, overwrite);
     this.rootTasks
         .add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(), preInsertTableDesc)));
-    TaskFactory
-        .getAndMakeChild(new DDLWork(getInputs(), getOutputs(), insertTableDesc), conf, tasks);
+
   }
 
+
   private void genAutoColumnStatsGatheringPipeline(QB qb, TableDesc table_desc,
                                                    Map<String, String> partSpec, Operator curr, boolean isInsertInto) throws SemanticException {
     String tableName = table_desc.getTableName();
@@ -12232,9 +12230,27 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
     }
     //find all Acid FileSinkOperatorS
     QueryPlanPostProcessor qp = new QueryPlanPostProcessor(rootTasks, acidFileSinks, ctx.getExecutionId());
+
+    // 10. Attach CTAS/Insert-Commit-hooks for Storage Handlers
+    final Optional<TezTask> optionalTezTask =
+        rootTasks.stream().filter(task -> task instanceof TezTask).map(task -> (TezTask) task)
+            .findFirst();
+    if (optionalTezTask.isPresent()) {
+      final TezTask tezTask = optionalTezTask.get();
+      rootTasks.stream()
+          .filter(task -> task.getWork() instanceof DDLWork)
+          .map(task -> (DDLWork) task.getWork())
+          .filter(ddlWork -> ddlWork.getPreInsertTableDesc() != null)
+          .map(ddlWork -> ddlWork.getPreInsertTableDesc())
+          .map(ddlPreInsertTask -> new InsertCommitHookDesc(ddlPreInsertTask.getTable(),
+              ddlPreInsertTask.isOverwrite()))
+          .forEach(insertCommitHookDesc -> tezTask.addDependentTask(
+              TaskFactory.get(new DDLWork(getInputs(), getOutputs(), insertCommitHookDesc), conf)));
+    }
+
     LOG.info("Completed plan generation");
 
-    // 10. put accessed columns to readEntity
+    // 11. put accessed columns to readEntity
     if (HiveConf.getBoolVar(this.conf, HiveConf.ConfVars.HIVE_STATS_COLLECT_SCANCOLS)) {
       putAccessedColumnsToReadEntity(inputs, columnAccessInfo);
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/b1357240/ql/src/java/org/apache/hadoop/hive/ql/parse/TaskCompiler.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/TaskCompiler.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/TaskCompiler.java
index df1d9cb..95e1c31 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/TaskCompiler.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/TaskCompiler.java
@@ -18,21 +18,6 @@
 
 package org.apache.hadoop.hive.ql.parse;
 
-import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.LinkedHashMap;
-import java.util.LinkedHashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
-import org.apache.hadoop.hive.ql.exec.DDLTask;
-import org.apache.hadoop.hive.ql.exec.MaterializedViewDesc;
-import org.apache.hadoop.hive.ql.io.AcidUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 import com.google.common.collect.Interner;
 import com.google.common.collect.Interners;
 import org.apache.hadoop.fs.Path;
@@ -43,7 +28,9 @@ import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.ql.Context;
 import org.apache.hadoop.hive.ql.ErrorMsg;
 import org.apache.hadoop.hive.ql.QueryState;
+import org.apache.hadoop.hive.ql.exec.DDLTask;
 import org.apache.hadoop.hive.ql.exec.FetchTask;
+import org.apache.hadoop.hive.ql.exec.MaterializedViewDesc;
 import org.apache.hadoop.hive.ql.exec.StatsTask;
 import org.apache.hadoop.hive.ql.exec.TableScanOperator;
 import org.apache.hadoop.hive.ql.exec.Task;
@@ -51,6 +38,7 @@ import org.apache.hadoop.hive.ql.exec.TaskFactory;
 import org.apache.hadoop.hive.ql.exec.Utilities;
 import org.apache.hadoop.hive.ql.hooks.ReadEntity;
 import org.apache.hadoop.hive.ql.hooks.WriteEntity;
+import org.apache.hadoop.hive.ql.io.AcidUtils;
 import org.apache.hadoop.hive.ql.io.orc.OrcInputFormat;
 import org.apache.hadoop.hive.ql.metadata.Hive;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
@@ -82,6 +70,18 @@ import org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe;
 import org.apache.hadoop.hive.serde2.thrift.ThriftFormatter;
 import org.apache.hadoop.hive.serde2.thrift.ThriftJDBCBinarySerDe;
 import org.apache.hadoop.mapred.InputFormat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
 
 /**
  * TaskCompiler is a the base class for classes that compile

http://git-wip-us.apache.org/repos/asf/hive/blob/b1357240/ql/src/java/org/apache/hadoop/hive/ql/plan/DDLWork.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/DDLWork.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/DDLWork.java
index 98da309..8ed3b03 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/DDLWork.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/DDLWork.java
@@ -17,15 +17,15 @@
  */
 package org.apache.hadoop.hive.ql.plan;
 
-import java.io.Serializable;
-import java.util.HashSet;
-
 import org.apache.hadoop.hive.ql.hooks.ReadEntity;
 import org.apache.hadoop.hive.ql.hooks.WriteEntity;
 import org.apache.hadoop.hive.ql.parse.AlterTablePartMergeFilesDesc;
 import org.apache.hadoop.hive.ql.parse.PreInsertTableDesc;
 import org.apache.hadoop.hive.ql.plan.Explain.Level;
 
+import java.io.Serializable;
+import java.util.HashSet;
+
 /**
  * DDLWork.
  *
@@ -35,7 +35,7 @@ public class DDLWork implements Serializable {
 
   // TODO: this can probably be replaced with much less code via dynamic dispatch and/or templates.
   private PreInsertTableDesc preInsertTableDesc;
-  private InsertTableDesc insertTableDesc;
+  private InsertCommitHookDesc insertCommitHookDesc;
   private AlterMaterializedViewDesc alterMVDesc;
   private CreateDatabaseDesc createDatabaseDesc;
   private SwitchDatabaseDesc switchDatabaseDesc;
@@ -522,9 +522,10 @@ public class DDLWork implements Serializable {
   }
 
   public DDLWork(HashSet<ReadEntity> inputs, HashSet<WriteEntity> outputs,
-          InsertTableDesc insertTableDesc) {
+          InsertCommitHookDesc insertCommitHookDesc
+  ) {
     this(inputs, outputs);
-    this.insertTableDesc = insertTableDesc;
+    this.insertCommitHookDesc = insertCommitHookDesc;
   }
 
   public DDLWork(HashSet<ReadEntity> inputs, HashSet<WriteEntity> outputs,
@@ -1241,12 +1242,12 @@ public class DDLWork implements Serializable {
   }
 
   @Explain(displayName = "Insert operator", explainLevels = { Level.USER, Level.DEFAULT, Level.EXTENDED })
-  public InsertTableDesc getInsertTableDesc() {
-    return insertTableDesc;
+  public InsertCommitHookDesc getInsertCommitHookDesc() {
+    return insertCommitHookDesc;
   }
 
-  public void setInsertTableDesc(InsertTableDesc insertTableDesc) {
-    this.insertTableDesc = insertTableDesc;
+  public void setInsertCommitHookDesc(InsertCommitHookDesc insertCommitHookDesc) {
+    this.insertCommitHookDesc = insertCommitHookDesc;
   }
 
   @Explain(displayName = "Pre Insert operator", explainLevels = { Level.USER, Level.DEFAULT, Level.EXTENDED })

http://git-wip-us.apache.org/repos/asf/hive/blob/b1357240/ql/src/java/org/apache/hadoop/hive/ql/plan/InsertCommitHookDesc.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/InsertCommitHookDesc.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/InsertCommitHookDesc.java
new file mode 100644
index 0000000..8136506
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/InsertCommitHookDesc.java
@@ -0,0 +1,41 @@
+/*
+ * 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.plan;
+
+import org.apache.hadoop.hive.ql.metadata.Table;
+
+@Explain(displayName = "Commit-Insert-Hook", explainLevels = { Explain.Level.USER,
+    Explain.Level.DEFAULT, Explain.Level.EXTENDED })
+public class InsertCommitHookDesc extends DDLDesc {
+  private final Table table;
+  private final boolean overwrite;
+
+  public InsertCommitHookDesc(Table table, boolean overwrite) {
+    this.table = table;
+    this.overwrite = overwrite;
+  }
+
+  public Table getTable() {
+    return table;
+  }
+
+  public boolean isOverwrite() {
+    return overwrite;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/b1357240/ql/src/java/org/apache/hadoop/hive/ql/plan/InsertTableDesc.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/InsertTableDesc.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/InsertTableDesc.java
deleted file mode 100644
index 212bc7a..0000000
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/InsertTableDesc.java
+++ /dev/null
@@ -1,40 +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.plan;
-
-import org.apache.hadoop.hive.ql.metadata.Table;
-
-@Explain(displayName = "Insert", explainLevels = { Explain.Level.USER, Explain.Level.DEFAULT, Explain.Level.EXTENDED })
-public class InsertTableDesc extends DDLDesc {
-  private final Table table;
-  private final boolean overwrite;
-
-  public InsertTableDesc(Table table, boolean overwrite) {
-    this.table = table;
-    this.overwrite = overwrite;
-  }
-
-  public Table getTable() {
-    return table;
-  }
-
-  public boolean isOverwrite() {
-    return overwrite;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/b1357240/ql/src/test/results/clientpositive/druid/druidmini_dynamic_partition.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/druid/druidmini_dynamic_partition.q.out b/ql/src/test/results/clientpositive/druid/druidmini_dynamic_partition.q.out
index 154e504..28deb79 100644
--- a/ql/src/test/results/clientpositive/druid/druidmini_dynamic_partition.q.out
+++ b/ql/src/test/results/clientpositive/druid/druidmini_dynamic_partition.q.out
@@ -323,9 +323,9 @@ SELECT cast (`ctimestamp2` as timestamp with local time zone) as `__time`,
 POSTHOOK: type: QUERY
 STAGE DEPENDENCIES:
   Stage-0 is a root stage
-  Stage-2
   Stage-1 is a root stage
-  Stage-3 is a root stage
+  Stage-2 is a root stage
+  Stage-4 depends on stages: Stage-2
 
 STAGE PLANS:
   Stage: Stage-0
@@ -336,15 +336,11 @@ STAGE PLANS:
           properties:
             COLUMN_STATS_ACCURATE 
 
-  Stage: Stage-2
-      Insert operator:
-        Insert
-
   Stage: Stage-1
       Pre Insert operator:
         Pre-Insert task
 
-  Stage: Stage-3
+  Stage: Stage-2
     Tez
 #### A masked pattern was here ####
       Edges:
@@ -389,6 +385,10 @@ STAGE PLANS:
                       serde: org.apache.hadoop.hive.druid.serde.DruidSerDe
                       name: default.druid_partitioned_table
 
+  Stage: Stage-4
+      Insert operator:
+        Commit-Insert-Hook
+
 PREHOOK: query: INSERT INTO TABLE druid_partitioned_table
 SELECT cast (`ctimestamp2` as timestamp with local time zone) as `__time`,
   cstring1,
@@ -460,9 +460,9 @@ POSTHOOK: query: EXPLAIN INSERT OVERWRITE TABLE druid_partitioned_table
 POSTHOOK: type: QUERY
 STAGE DEPENDENCIES:
   Stage-0 is a root stage
-  Stage-2
   Stage-1 is a root stage
-  Stage-3 is a root stage
+  Stage-2 is a root stage
+  Stage-4 depends on stages: Stage-2
 
 STAGE PLANS:
   Stage: Stage-0
@@ -473,15 +473,11 @@ STAGE PLANS:
           properties:
             COLUMN_STATS_ACCURATE 
 
-  Stage: Stage-2
-      Insert operator:
-        Insert
-
   Stage: Stage-1
       Pre Insert operator:
         Pre-Insert task
 
-  Stage: Stage-3
+  Stage: Stage-2
     Tez
 #### A masked pattern was here ####
       Edges:
@@ -526,6 +522,10 @@ STAGE PLANS:
                       serde: org.apache.hadoop.hive.druid.serde.DruidSerDe
                       name: default.druid_partitioned_table
 
+  Stage: Stage-4
+      Insert operator:
+        Commit-Insert-Hook
+
 PREHOOK: query: INSERT OVERWRITE TABLE druid_partitioned_table
   SELECT cast (`ctimestamp1` as timestamp with local time zone) as `__time`,
     cstring1,

http://git-wip-us.apache.org/repos/asf/hive/blob/b1357240/ql/src/test/results/clientpositive/druid/druidmini_mv.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/druid/druidmini_mv.q.out b/ql/src/test/results/clientpositive/druid/druidmini_mv.q.out
index c29f547..a5d338f 100644
--- a/ql/src/test/results/clientpositive/druid/druidmini_mv.q.out
+++ b/ql/src/test/results/clientpositive/druid/druidmini_mv.q.out
@@ -422,10 +422,10 @@ ALTER MATERIALIZED VIEW cmv_mat_view2 REBUILD
 POSTHOOK: type: QUERY
 STAGE DEPENDENCIES:
   Stage-0 is a root stage
-  Stage-2
-  Stage-5 depends on stages: Stage-2, Stage-1, Stage-3
+  Stage-4 depends on stages: Stage-0, Stage-1, Stage-2
   Stage-1 is a root stage
-  Stage-3 is a root stage
+  Stage-2 is a root stage
+  Stage-5 depends on stages: Stage-2
 
 STAGE PLANS:
   Stage: Stage-0
@@ -436,18 +436,14 @@ STAGE PLANS:
           properties:
             COLUMN_STATS_ACCURATE 
 
-  Stage: Stage-2
-      Insert operator:
-        Insert
-
-  Stage: Stage-5
+  Stage: Stage-4
     Materialized View Work
 
   Stage: Stage-1
       Pre Insert operator:
         Pre-Insert task
 
-  Stage: Stage-3
+  Stage: Stage-2
     Tez
 #### A masked pattern was here ####
       Edges:
@@ -492,6 +488,10 @@ STAGE PLANS:
                       serde: org.apache.hadoop.hive.druid.serde.DruidSerDe
                       name: default.cmv_mat_view2
 
+  Stage: Stage-5
+      Insert operator:
+        Commit-Insert-Hook
+
 PREHOOK: query: ALTER MATERIALIZED VIEW cmv_mat_view2 REBUILD
 PREHOOK: type: QUERY
 PREHOOK: Input: default@cmv_basetable

http://git-wip-us.apache.org/repos/asf/hive/blob/b1357240/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/DefaultHiveMetaHook.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/DefaultHiveMetaHook.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/DefaultHiveMetaHook.java
index 0957945..4fc913c 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/DefaultHiveMetaHook.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/DefaultHiveMetaHook.java
@@ -23,7 +23,7 @@ import org.apache.hadoop.hive.metastore.api.Table;
 
 public abstract class DefaultHiveMetaHook implements HiveMetaHook {
   /**
-   * Called after successfully after INSERT [OVERWRITE] statement is executed.
+   * Called after successfully INSERT [OVERWRITE] statement is executed.
    * @param table table definition
    * @param overwrite true if it is INSERT OVERWRITE
    *


[24/50] [abbrv] hive git commit: HIVE-19476: Fix failures in TestReplicationScenariosAcidTables, TestReplicationOnHDFSEncryptedZones and TestCopyUtils (Sankar Hariappan, reviewed by Sergey Shelukhin)

Posted by vg...@apache.org.
HIVE-19476: Fix failures in TestReplicationScenariosAcidTables, TestReplicationOnHDFSEncryptedZones and TestCopyUtils (Sankar Hariappan, reviewed by Sergey Shelukhin)


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

Branch: refs/heads/branch-3.0.0
Commit: 1b917c2048d704d68f121453b96fb7fe717668e4
Parents: 91b66c5
Author: Sankar Hariappan <sa...@apache.org>
Authored: Mon May 14 00:40:22 2018 +0530
Committer: Sankar Hariappan <sa...@apache.org>
Committed: Mon May 14 00:40:22 2018 +0530

----------------------------------------------------------------------
 .../hive/ql/parse/TestReplicationOnHDFSEncryptedZones.java      | 5 ++++-
 .../java/org/apache/hadoop/hive/ql/parse/WarehouseInstance.java | 1 +
 2 files changed, 5 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/1b917c20/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationOnHDFSEncryptedZones.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationOnHDFSEncryptedZones.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationOnHDFSEncryptedZones.java
index fd05e99..d8d210b 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationOnHDFSEncryptedZones.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationOnHDFSEncryptedZones.java
@@ -94,6 +94,8 @@ public class TestReplicationOnHDFSEncryptedZones {
         new HashMap<String, String>() {{
           put(HiveConf.ConfVars.HIVE_IN_TEST.varname, "false");
           put(HiveConf.ConfVars.HIVE_SERVER2_ENABLE_DOAS.varname, "false");
+          put(HiveConf.ConfVars.HIVE_DISTCP_DOAS_USER.varname,
+                  UserGroupInformation.getCurrentUser().getUserName());
         }}, "test_key123");
 
     WarehouseInstance.Tuple tuple =
@@ -105,7 +107,8 @@ public class TestReplicationOnHDFSEncryptedZones {
 
     replica
         .run("repl load " + replicatedDbName + " from '" + tuple.dumpLocation
-            + "' with('hive.repl.add.raw.reserved.namespace'='true')")
+                + "' with('hive.repl.add.raw.reserved.namespace'='true', "
+                + "'distcp.options.pugpbx'='', 'distcp.options.skipcrccheck'='', 'distcp.options.update'='')")
         .run("use " + replicatedDbName)
         .run("repl status " + replicatedDbName)
         .verifyResult(tuple.lastReplicationId)

http://git-wip-us.apache.org/repos/asf/hive/blob/1b917c20/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/WarehouseInstance.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/WarehouseInstance.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/WarehouseInstance.java
index accdc1f..85a7e1b 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/WarehouseInstance.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/WarehouseInstance.java
@@ -123,6 +123,7 @@ public class WarehouseInstance implements Closeable {
     hiveConf.setIntVar(HiveConf.ConfVars.METASTORETHRIFTCONNECTIONRETRIES, 3);
     hiveConf.set(HiveConf.ConfVars.PREEXECHOOKS.varname, "");
     hiveConf.set(HiveConf.ConfVars.POSTEXECHOOKS.varname, "");
+    hiveConf.setBoolVar(HiveConf.ConfVars.HIVESTATSAUTOGATHER, false);
     if (!hiveConf.getVar(HiveConf.ConfVars.HIVE_TXN_MANAGER).equals("org.apache.hadoop.hive.ql.lockmgr.DbTxnManager")) {
       hiveConf.set(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY.varname, "false");
     }


[14/50] [abbrv] hive git commit: HIVE-19135 Need tool to allow admins to create catalogs and move existing dbs to catalog during upgrade (Alan Gates, reviewed by Thejas Nair)

Posted by vg...@apache.org.
HIVE-19135 Need tool to allow admins to create catalogs and move existing dbs to catalog during upgrade (Alan Gates, reviewed by Thejas Nair)


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

Branch: refs/heads/branch-3.0.0
Commit: 5bb3df0eec9fd80f921c7bfb7a2ffbc40312bd58
Parents: a8fc0e6
Author: Alan Gates <ga...@hortonworks.com>
Authored: Tue May 1 12:29:55 2018 -0700
Committer: Vineet Garg <vg...@apache.org>
Committed: Wed May 9 13:32:07 2018 -0700

----------------------------------------------------------------------
 .../org/apache/hive/beeline/HiveSchemaTool.java | 275 +++++++++++++-
 .../hive/beeline/TestSchemaToolCatalogOps.java  | 375 +++++++++++++++++++
 .../hadoop/hive/metastore/ObjectStore.java      |   1 +
 3 files changed, 649 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/5bb3df0e/beeline/src/java/org/apache/hive/beeline/HiveSchemaTool.java
----------------------------------------------------------------------
diff --git a/beeline/src/java/org/apache/hive/beeline/HiveSchemaTool.java b/beeline/src/java/org/apache/hive/beeline/HiveSchemaTool.java
index a90127b..a469cd4 100644
--- a/beeline/src/java/org/apache/hive/beeline/HiveSchemaTool.java
+++ b/beeline/src/java/org/apache/hive/beeline/HiveSchemaTool.java
@@ -17,6 +17,7 @@
  */
 package org.apache.hive.beeline;
 
+import com.google.common.annotations.VisibleForTesting;
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.CommandLineParser;
 import org.apache.commons.cli.GnuParser;
@@ -33,6 +34,7 @@ import org.apache.commons.lang3.tuple.Pair;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
+import org.apache.hadoop.hive.metastore.DatabaseProduct;
 import org.apache.hadoop.hive.metastore.HiveMetaException;
 import org.apache.hadoop.hive.metastore.IMetaStoreSchemaInfo;
 import org.apache.hadoop.hive.metastore.MetaStoreSchemaInfoFactory;
@@ -42,6 +44,7 @@ import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
 import org.apache.hadoop.hive.metastore.tools.HiveSchemaHelper;
 import org.apache.hadoop.hive.metastore.tools.HiveSchemaHelper.MetaStoreConnectionInfo;
 import org.apache.hadoop.hive.metastore.tools.HiveSchemaHelper.NestedScriptParser;
+import org.apache.hadoop.hive.metastore.tools.SQLGenerator;
 import org.apache.hadoop.hive.shims.ShimLoader;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -71,6 +74,8 @@ import java.util.Map;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
+import static org.apache.hadoop.hive.metastore.utils.StringUtils.normalizeIdentifier;
+
 public class HiveSchemaTool {
   private String userName = null;
   private String passWord = null;
@@ -85,6 +90,7 @@ public class HiveSchemaTool {
   private final String metaDbType;
   private final IMetaStoreSchemaInfo metaStoreSchemaInfo;
   private boolean needsQuotedIdentifier;
+  private String quoteCharacter;
 
   static final private Logger LOG = LoggerFactory.getLogger(HiveSchemaTool.class.getName());
 
@@ -100,7 +106,9 @@ public class HiveSchemaTool {
     this.hiveConf = hiveConf;
     this.dbType = dbType;
     this.metaDbType = metaDbType;
-    this.needsQuotedIdentifier = getDbCommandParser(dbType, metaDbType).needsQuotedIdentifier();
+    NestedScriptParser parser = getDbCommandParser(dbType, metaDbType);
+    this.needsQuotedIdentifier = parser.needsQuotedIdentifier();
+    this.quoteCharacter = parser.getQuoteCharacter();
     this.metaStoreSchemaInfo = MetaStoreSchemaInfoFactory.get(hiveConf, hiveHome, dbType);
   }
 
@@ -878,6 +886,204 @@ public class HiveSchemaTool {
     }
   }
 
+  @VisibleForTesting
+  void createCatalog(String catName, String location, String description, boolean ifNotExists)
+      throws HiveMetaException {
+    catName = normalizeIdentifier(catName);
+    System.out.println("Create catalog " + catName + " at location " + location);
+
+    Connection conn = getConnectionToMetastore(true);
+    boolean success = false;
+    try {
+      conn.setAutoCommit(false);
+      try (Statement stmt = conn.createStatement()) {
+        // If they set ifNotExists check for existence first, and bail if it exists.  This is
+        // more reliable then attempting to parse the error message from the SQLException.
+        if (ifNotExists) {
+          String query = "select " + quoteIf("NAME") + " from " + quoteIf("CTLGS") +
+              " where " + quoteIf("NAME") + " = '" + catName + "'";
+          LOG.debug("Going to run " + query);
+          ResultSet rs = stmt.executeQuery(query);
+          if (rs.next()) {
+            System.out.println("Catalog " + catName + " already exists");
+            return;
+          }
+        }
+        SQLGenerator sqlGenerator = new SQLGenerator(
+            DatabaseProduct.determineDatabaseProduct(
+                conn.getMetaData().getDatabaseProductName()
+            ), hiveConf);
+        String query = sqlGenerator.addForUpdateClause("select max(" + quoteIf("CTLG_ID") + ") " +
+            "from " + quoteIf("CTLGS"));
+        LOG.debug("Going to run " + query);
+        ResultSet rs = stmt.executeQuery(query);
+        if (!rs.next()) {
+          throw new HiveMetaException("No catalogs found, have you upgraded the database?");
+        }
+        int catNum = rs.getInt(1) + 1;
+
+        String update = "insert into " + quoteIf("CTLGS") +
+            "(" + quoteIf("CTLG_ID") + ", " + quoteIf("NAME") + ", " + quoteAlways("DESC") + ", " + quoteIf( "LOCATION_URI") + ") " +
+            " values (" + catNum + ", '" + catName + "', '" + description + "', '" + location + "')";
+        LOG.debug("Going to run " + update);
+        stmt.execute(update);
+        conn.commit();
+        success = true;
+      }
+    } catch (MetaException|SQLException e) {
+      throw new HiveMetaException("Failed to add catalog", e);
+    } finally {
+      try {
+        if (!success) conn.rollback();
+      } catch (SQLException e) {
+        // Not really much we can do here.
+        LOG.error("Failed to rollback, everything will probably go bad from here.");
+      }
+    }
+  }
+
+  @VisibleForTesting
+  void moveDatabase(String fromCatName, String toCatName, String dbName) throws HiveMetaException {
+    fromCatName = normalizeIdentifier(fromCatName);
+    toCatName = normalizeIdentifier(toCatName);
+    dbName = normalizeIdentifier(dbName);
+    System.out.println("Moving database " + dbName + " from catalog " + fromCatName +
+        " to catalog " + toCatName);
+    Connection conn = getConnectionToMetastore(true);
+    boolean success = false;
+    try {
+      conn.setAutoCommit(false);
+      try (Statement stmt = conn.createStatement()) {
+        updateCatalogNameInTable(stmt, "DBS", "CTLG_NAME", "NAME", fromCatName, toCatName, dbName, false);
+        updateCatalogNameInTable(stmt, "TAB_COL_STATS", "CAT_NAME", "DB_NAME", fromCatName, toCatName, dbName, true);
+        updateCatalogNameInTable(stmt, "PART_COL_STATS", "CAT_NAME", "DB_NAME", fromCatName, toCatName, dbName, true);
+        updateCatalogNameInTable(stmt, "PARTITION_EVENTS", "CAT_NAME", "DB_NAME", fromCatName, toCatName, dbName, true);
+        updateCatalogNameInTable(stmt, "NOTIFICATION_LOG", "CAT_NAME", "DB_NAME", fromCatName, toCatName, dbName, true);
+        conn.commit();
+        success = true;
+      }
+    } catch (SQLException e) {
+      throw new HiveMetaException("Failed to move database", e);
+    } finally {
+      try {
+        if (!success) conn.rollback();
+      } catch (SQLException e) {
+        // Not really much we can do here.
+        LOG.error("Failed to rollback, everything will probably go bad from here.");
+      }
+    }
+  }
+
+  private void updateCatalogNameInTable(Statement stmt, String tableName, String catColName,
+                                        String dbColName, String fromCatName,
+                                        String toCatName, String dbName, boolean zeroUpdatesOk)
+      throws HiveMetaException, SQLException {
+    String update = "update " + quoteIf(tableName) + " " +
+        "set " + quoteIf(catColName) + " = '" + toCatName + "' " +
+        "where " + quoteIf(catColName) + " = '" + fromCatName + "' and " + quoteIf(dbColName) + " = '" + dbName + "'";
+    LOG.debug("Going to run " + update);
+    int numUpdated = stmt.executeUpdate(update);
+    if (numUpdated != 1 && !(zeroUpdatesOk && numUpdated == 0)) {
+      throw new HiveMetaException("Failed to properly update the " + tableName +
+          " table.  Expected to update 1 row but instead updated " + numUpdated);
+    }
+  }
+
+  @VisibleForTesting
+  void moveTable(String fromCat, String toCat, String fromDb, String toDb, String tableName)
+      throws HiveMetaException {
+    fromCat = normalizeIdentifier(fromCat);
+    toCat = normalizeIdentifier(toCat);
+    fromDb = normalizeIdentifier(fromDb);
+    toDb = normalizeIdentifier(toDb);
+    tableName = normalizeIdentifier(tableName);
+    Connection conn = getConnectionToMetastore(true);
+    boolean success = false;
+    try {
+      conn.setAutoCommit(false);
+      try (Statement stmt = conn.createStatement()) {
+        // Find the old database id
+        String query = "select " + quoteIf("DB_ID") +
+            " from " + quoteIf("DBS") +
+            " where " + quoteIf("NAME") + " = '" + fromDb + "' "
+                + "and " + quoteIf("CTLG_NAME") + " = '" + fromCat + "'";
+        LOG.debug("Going to run " + query);
+        ResultSet rs = stmt.executeQuery(query);
+        if (!rs.next()) {
+          throw new HiveMetaException("Unable to find database " + fromDb);
+        }
+        long oldDbId = rs.getLong(1);
+
+        // Find the new database id
+        query = "select " + quoteIf("DB_ID") +
+            " from " + quoteIf("DBS") +
+            " where " + quoteIf("NAME") + " = '" + toDb + "' "
+                + "and " + quoteIf("CTLG_NAME") + " = '" + toCat + "'";
+        LOG.debug("Going to run " + query);
+        rs = stmt.executeQuery(query);
+        if (!rs.next()) {
+          throw new HiveMetaException("Unable to find database " + toDb);
+        }
+        long newDbId = rs.getLong(1);
+
+        String update = "update " + quoteIf("TBLS") + " " +
+            "set " + quoteIf("DB_ID") + " = " + newDbId + " " +
+            "where " + quoteIf("DB_ID") + " = " + oldDbId +
+                " and " + quoteIf("TBL_NAME") + " = '" + tableName + "'";
+        LOG.debug("Going to run " + update);
+        int numUpdated = stmt.executeUpdate(update);
+        if (numUpdated != 1) {
+          throw new HiveMetaException(
+              "Failed to properly update TBLS table.  Expected to update " +
+                  "1 row but instead updated " + numUpdated);
+        }
+        updateDbNameForTable(stmt, "TAB_COL_STATS", "TABLE_NAME", fromCat, toCat, fromDb, toDb, tableName);
+        updateDbNameForTable(stmt, "PART_COL_STATS", "TABLE_NAME", fromCat, toCat, fromDb, toDb, tableName);
+        updateDbNameForTable(stmt, "PARTITION_EVENTS", "TBL_NAME", fromCat, toCat, fromDb, toDb, tableName);
+        updateDbNameForTable(stmt, "NOTIFICATION_LOG", "TBL_NAME", fromCat, toCat, fromDb, toDb, tableName);
+        conn.commit();
+        success = true;
+      }
+    } catch (SQLException se) {
+      throw new HiveMetaException("Failed to move table", se);
+    } finally {
+      try {
+        if (!success) conn.rollback();
+      } catch (SQLException e) {
+        // Not really much we can do here.
+        LOG.error("Failed to rollback, everything will probably go bad from here.");
+      }
+
+    }
+  }
+
+  private void updateDbNameForTable(Statement stmt, String tableName,
+                                    String tableColumnName, String fromCat, String toCat,
+                                    String fromDb, String toDb, String hiveTblName)
+      throws HiveMetaException, SQLException {
+    String update = "update " + quoteIf(tableName) + " " +
+            "set " + quoteIf("CAT_NAME") + " = '" + toCat + "', " + quoteIf("DB_NAME") + " = '" + toDb + "' " +
+            "where " + quoteIf("CAT_NAME") + " = '" + fromCat + "' " +
+                "and " + quoteIf("DB_NAME") + " = '" + fromDb + "' " +
+                "and " + quoteIf(tableColumnName) + " = '" + hiveTblName + "'";
+    LOG.debug("Going to run " + update);
+    int numUpdated = stmt.executeUpdate(update);
+    if (numUpdated > 1 || numUpdated < 0) {
+      throw new HiveMetaException("Failed to properly update the " + tableName +
+          " table.  Expected to update 1 row but instead updated " + numUpdated);
+    }
+  }
+
+  // Quote if the database requires it
+  private String quoteIf(String identifier) {
+    return needsQuotedIdentifier ? quoteCharacter + identifier + quoteCharacter : identifier;
+  }
+
+  // Quote always, for fields that mimic SQL keywords, like DESC
+  private String quoteAlways(String identifier) {
+    return quoteCharacter + identifier + quoteCharacter;
+  }
+
   /**
    *  Run pre-upgrade scripts corresponding to a given upgrade script,
    *  if any exist. The errors from pre-upgrade are ignored.
@@ -1026,11 +1232,27 @@ public class HiveSchemaTool {
                 create("initSchemaTo");
     Option infoOpt = new Option("info", "Show config and schema details");
     Option validateOpt = new Option("validate", "Validate the database");
+    Option createCatalog = OptionBuilder
+        .hasArg()
+        .withDescription("Create a catalog, requires --catalogLocation parameter as well")
+        .create("createCatalog");
+    Option moveDatabase = OptionBuilder
+        .hasArg()
+        .withDescription("Move a database between catalogs.  Argument is the database name. " +
+            "Requires --fromCatalog and --toCatalog parameters as well")
+        .create("moveDatabase");
+    Option moveTable = OptionBuilder
+        .hasArg()
+        .withDescription("Move a table to a different database.  Argument is the table name. " +
+            "Requires --fromCatalog, --toCatalog, --fromDatabase, and --toDatabase " +
+            " parameters as well.")
+        .create("moveTable");
 
     OptionGroup optGroup = new OptionGroup();
     optGroup.addOption(upgradeOpt).addOption(initOpt).
                 addOption(help).addOption(upgradeFromOpt).
-                addOption(initToOpt).addOption(infoOpt).addOption(validateOpt);
+                addOption(initToOpt).addOption(infoOpt).addOption(validateOpt)
+                .addOption(createCatalog).addOption(moveDatabase).addOption(moveTable);
     optGroup.setRequired(true);
 
     Option userNameOpt = OptionBuilder.withArgName("user")
@@ -1061,6 +1283,37 @@ public class HiveSchemaTool {
     Option serversOpt = OptionBuilder.withArgName("serverList")
         .hasArgs().withDescription("a comma-separated list of servers used in location validation in the format of scheme://authority (e.g. hdfs://localhost:8000)")
         .create("servers");
+    Option catalogLocation = OptionBuilder
+        .hasArg()
+        .withDescription("Location of new catalog, required when adding a catalog")
+        .create("catalogLocation");
+    Option catalogDescription = OptionBuilder
+        .hasArg()
+        .withDescription("Description of new catalog")
+        .create("catalogDescription");
+    Option ifNotExists = OptionBuilder
+        .withDescription("If passed then it is not an error to create an existing catalog")
+        .create("ifNotExists");
+    Option toCatalog = OptionBuilder
+        .hasArg()
+        .withDescription("Catalog a moving database or table is going to.  This is " +
+            "required if you are moving a database or table.")
+        .create("toCatalog");
+    Option fromCatalog = OptionBuilder
+        .hasArg()
+        .withDescription("Catalog a moving database or table is coming from.  This is " +
+            "required if you are moving a database or table.")
+        .create("fromCatalog");
+    Option toDatabase = OptionBuilder
+        .hasArg()
+        .withDescription("Database a moving table is going to.  This is " +
+            "required if you are moving a table.")
+        .create("toDatabase");
+    Option fromDatabase = OptionBuilder
+        .hasArg()
+        .withDescription("Database a moving table is coming from.  This is " +
+            "required if you are moving a table.")
+        .create("fromDatabase");
     cmdLineOptions.addOption(help);
     cmdLineOptions.addOption(dryRunOpt);
     cmdLineOptions.addOption(userNameOpt);
@@ -1072,6 +1325,13 @@ public class HiveSchemaTool {
     cmdLineOptions.addOption(driverOpt);
     cmdLineOptions.addOption(dbOpts);
     cmdLineOptions.addOption(serversOpt);
+    cmdLineOptions.addOption(catalogLocation);
+    cmdLineOptions.addOption(catalogDescription);
+    cmdLineOptions.addOption(ifNotExists);
+    cmdLineOptions.addOption(toCatalog);
+    cmdLineOptions.addOption(fromCatalog);
+    cmdLineOptions.addOption(toDatabase);
+    cmdLineOptions.addOption(fromDatabase);
     cmdLineOptions.addOptionGroup(optGroup);
   }
 
@@ -1188,6 +1448,17 @@ public class HiveSchemaTool {
         schemaTool.doInit(schemaVer);
       } else if (line.hasOption("validate")) {
         schemaTool.doValidate();
+      } else if (line.hasOption("createCatalog")) {
+        schemaTool.createCatalog(line.getOptionValue("createCatalog"),
+            line.getOptionValue("catalogLocation"), line.getOptionValue("catalogDescription"),
+            line.hasOption("ifNotExists"));
+      } else if (line.hasOption("moveDatabase")) {
+        schemaTool.moveDatabase(line.getOptionValue("fromCatalog"),
+            line.getOptionValue("toCatalog"), line.getOptionValue("moveDatabase"));
+      } else if (line.hasOption("moveTable")) {
+        schemaTool.moveTable(line.getOptionValue("fromCatalog"), line.getOptionValue("toCatalog"),
+            line.getOptionValue("fromDatabase"), line.getOptionValue("toDatabase"),
+            line.getOptionValue("moveTable"));
       } else {
         System.err.println("no valid option supplied");
         printAndExit(cmdLineOptions);

http://git-wip-us.apache.org/repos/asf/hive/blob/5bb3df0e/itests/hive-unit/src/test/java/org/apache/hive/beeline/TestSchemaToolCatalogOps.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hive/beeline/TestSchemaToolCatalogOps.java b/itests/hive-unit/src/test/java/org/apache/hive/beeline/TestSchemaToolCatalogOps.java
new file mode 100644
index 0000000..db10ae6
--- /dev/null
+++ b/itests/hive-unit/src/test/java/org/apache/hive/beeline/TestSchemaToolCatalogOps.java
@@ -0,0 +1,375 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hive.beeline;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.HiveMetaException;
+import org.apache.hadoop.hive.metastore.HiveMetaStoreClient;
+import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.metastore.api.Catalog;
+import org.apache.hadoop.hive.metastore.api.Database;
+import org.apache.hadoop.hive.metastore.api.Function;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.client.builder.CatalogBuilder;
+import org.apache.hadoop.hive.metastore.client.builder.DatabaseBuilder;
+import org.apache.hadoop.hive.metastore.client.builder.FunctionBuilder;
+import org.apache.hadoop.hive.metastore.client.builder.PartitionBuilder;
+import org.apache.hadoop.hive.metastore.client.builder.TableBuilder;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import org.apache.thrift.TException;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Set;
+
+import static org.apache.hadoop.hive.metastore.Warehouse.DEFAULT_CATALOG_NAME;
+import static org.apache.hadoop.hive.metastore.Warehouse.DEFAULT_DATABASE_NAME;
+
+public class TestSchemaToolCatalogOps {
+  private static final Logger LOG = LoggerFactory.getLogger(TestSchemaToolCatalogOps.class);
+  private static HiveSchemaTool schemaTool;
+  private static HiveConf conf;
+  private IMetaStoreClient client;
+  private static String testMetastoreDB;
+
+  @BeforeClass
+  public static void initDb() throws HiveMetaException, IOException {
+    conf = new HiveConf();
+    MetastoreConf.setBoolVar(conf, MetastoreConf.ConfVars.AUTO_CREATE_ALL, false);
+    MetastoreConf.setLongVar(conf, MetastoreConf.ConfVars.HMS_HANDLER_ATTEMPTS, 1);
+    MetastoreConf.setLongVar(conf, MetastoreConf.ConfVars.THRIFT_CONNECTION_RETRIES, 1);
+    testMetastoreDB = System.getProperty("java.io.tmpdir") +
+        File.separator + "testschematoolcatopsdb";
+    MetastoreConf.setVar(conf, MetastoreConf.ConfVars.CONNECT_URL_KEY,
+        "jdbc:derby:" + testMetastoreDB + ";create=true");
+    schemaTool = new HiveSchemaTool(
+        System.getProperty("test.tmp.dir", "target/tmp"), conf, "derby", null);
+    schemaTool.setUserName(MetastoreConf.getVar(conf, MetastoreConf.ConfVars.CONNECTION_USER_NAME));
+    schemaTool.setPassWord(MetastoreConf.getPassword(conf, MetastoreConf.ConfVars.PWD));
+    schemaTool.doInit();  // Pre-install the database so all the tables are there.
+
+  }
+
+  @AfterClass
+  public static void removeDb() throws Exception {
+    File metaStoreDir = new File(testMetastoreDB);
+    if (metaStoreDir.exists()) {
+      FileUtils.forceDeleteOnExit(metaStoreDir);
+    }
+  }
+
+  @Before
+  public void createClient() throws MetaException {
+    client = new HiveMetaStoreClient(conf);
+  }
+
+  @Test
+  public void createCatalog() throws HiveMetaException, TException {
+    String catName = "my_test_catalog";
+    String location = "file:///tmp/my_test_catalog";
+    String description = "very descriptive";
+    schemaTool.createCatalog(catName, location, description, false);
+
+    Catalog cat = client.getCatalog(catName);
+    Assert.assertEquals(location, cat.getLocationUri());
+    Assert.assertEquals(description, cat.getDescription());
+  }
+
+  @Test(expected = HiveMetaException.class)
+  public void createExistingCatalog() throws HiveMetaException {
+    schemaTool.createCatalog("hive", "somewhere", "", false);
+  }
+
+  @Test
+  public void createExistingCatalogWithIfNotExists() throws HiveMetaException, TException {
+    String catName = "my_existing_test_catalog";
+    String location = "file:///tmp/my_test_catalog";
+    String description = "very descriptive";
+    schemaTool.createCatalog(catName, location, description, false);
+
+    schemaTool.createCatalog(catName, location, description, true);
+  }
+
+  @Test
+  public void moveDatabase() throws HiveMetaException, TException {
+    String toCatName = "moveDbCat";
+    String dbName = "moveDbDb";
+    String tableName = "moveDbTable";
+    String funcName = "movedbfunc";
+    String partVal = "moveDbKey";
+
+    new CatalogBuilder()
+        .setName(toCatName)
+        .setLocation("file:///tmp")
+        .create(client);
+
+    Database db = new DatabaseBuilder()
+        .setCatalogName(DEFAULT_CATALOG_NAME)
+        .setName(dbName)
+        .create(client, conf);
+
+    new FunctionBuilder()
+        .inDb(db)
+        .setName(funcName)
+        .setClass("org.apache.hive.myudf")
+        .create(client, conf);
+
+    Table table = new TableBuilder()
+        .inDb(db)
+        .setTableName(tableName)
+        .addCol("a", "int")
+        .addPartCol("p", "string")
+        .create(client, conf);
+
+    new PartitionBuilder()
+        .inTable(table)
+        .addValue(partVal)
+        .addToTable(client, conf);
+
+    schemaTool.moveDatabase(DEFAULT_CATALOG_NAME, toCatName, dbName);
+
+    Database fetchedDb = client.getDatabase(toCatName, dbName);
+    Assert.assertNotNull(fetchedDb);
+    Assert.assertEquals(toCatName.toLowerCase(), fetchedDb.getCatalogName());
+
+    Function fetchedFunction = client.getFunction(toCatName, dbName, funcName);
+    Assert.assertNotNull(fetchedFunction);
+    Assert.assertEquals(toCatName.toLowerCase(), fetchedFunction.getCatName());
+    Assert.assertEquals(dbName.toLowerCase(), fetchedFunction.getDbName());
+
+    Table fetchedTable = client.getTable(toCatName, dbName, tableName);
+    Assert.assertNotNull(fetchedTable);
+    Assert.assertEquals(toCatName.toLowerCase(), fetchedTable.getCatName());
+    Assert.assertEquals(dbName.toLowerCase(), fetchedTable.getDbName());
+
+    Partition fetchedPart =
+        client.getPartition(toCatName, dbName, tableName, Collections.singletonList(partVal));
+    Assert.assertNotNull(fetchedPart);
+    Assert.assertEquals(toCatName.toLowerCase(), fetchedPart.getCatName());
+    Assert.assertEquals(dbName.toLowerCase(), fetchedPart.getDbName());
+    Assert.assertEquals(tableName.toLowerCase(), fetchedPart.getTableName());
+  }
+
+  @Test
+  public void moveDatabaseWithExistingDbOfSameNameAlreadyInTargetCatalog()
+      throws TException, HiveMetaException {
+    String catName = "clobberCatalog";
+    new CatalogBuilder()
+        .setName(catName)
+        .setLocation("file:///tmp")
+        .create(client);
+    try {
+      schemaTool.moveDatabase(catName, DEFAULT_CATALOG_NAME, DEFAULT_DATABASE_NAME);
+      Assert.fail("Attempt to move default database should have failed.");
+    } catch (HiveMetaException e) {
+      // good
+    }
+
+    // Make sure nothing really moved
+    Set<String> dbNames = new HashSet<>(client.getAllDatabases(DEFAULT_CATALOG_NAME));
+    Assert.assertTrue(dbNames.contains(DEFAULT_DATABASE_NAME));
+  }
+
+  @Test(expected = HiveMetaException.class)
+  public void moveNonExistentDatabase() throws TException, HiveMetaException {
+    String catName = "moveNonExistentDb";
+    new CatalogBuilder()
+        .setName(catName)
+        .setLocation("file:///tmp")
+        .create(client);
+    schemaTool.moveDatabase(catName, DEFAULT_CATALOG_NAME, "nosuch");
+  }
+
+  @Test
+  public void moveDbToNonExistentCatalog() throws TException, HiveMetaException {
+    String dbName = "doomedToHomelessness";
+    new DatabaseBuilder()
+        .setName(dbName)
+        .create(client, conf);
+    try {
+      schemaTool.moveDatabase(DEFAULT_CATALOG_NAME, "nosuch", dbName);
+      Assert.fail("Attempt to move database to non-existent catalog should have failed.");
+    } catch (HiveMetaException e) {
+      // good
+    }
+
+    // Make sure nothing really moved
+    Set<String> dbNames = new HashSet<>(client.getAllDatabases(DEFAULT_CATALOG_NAME));
+    Assert.assertTrue(dbNames.contains(dbName.toLowerCase()));
+  }
+
+  @Test
+  public void moveTable() throws TException, HiveMetaException {
+    String toCatName = "moveTableCat";
+    String toDbName = "moveTableDb";
+    String tableName = "moveTableTable";
+    String partVal = "moveTableKey";
+
+    new CatalogBuilder()
+        .setName(toCatName)
+        .setLocation("file:///tmp")
+        .create(client);
+
+    new DatabaseBuilder()
+        .setCatalogName(toCatName)
+        .setName(toDbName)
+        .create(client, conf);
+
+    Table table = new TableBuilder()
+        .setTableName(tableName)
+        .addCol("a", "int")
+        .addPartCol("p", "string")
+        .create(client, conf);
+
+    new PartitionBuilder()
+        .inTable(table)
+        .addValue(partVal)
+        .addToTable(client, conf);
+
+    schemaTool.moveTable(DEFAULT_CATALOG_NAME, toCatName, DEFAULT_DATABASE_NAME, toDbName, tableName);
+
+    Table fetchedTable = client.getTable(toCatName, toDbName, tableName);
+    Assert.assertNotNull(fetchedTable);
+    Assert.assertEquals(toCatName.toLowerCase(), fetchedTable.getCatName());
+    Assert.assertEquals(toDbName.toLowerCase(), fetchedTable.getDbName());
+
+    Partition fetchedPart =
+        client.getPartition(toCatName, toDbName, tableName, Collections.singletonList(partVal));
+    Assert.assertNotNull(fetchedPart);
+    Assert.assertEquals(toCatName.toLowerCase(), fetchedPart.getCatName());
+    Assert.assertEquals(toDbName.toLowerCase(), fetchedPart.getDbName());
+    Assert.assertEquals(tableName.toLowerCase(), fetchedPart.getTableName());
+  }
+
+  @Test
+  public void moveTableWithinCatalog() throws TException, HiveMetaException {
+    String toDbName = "moveTableWithinCatalogDb";
+    String tableName = "moveTableWithinCatalogTable";
+    String partVal = "moveTableWithinCatalogKey";
+
+    new DatabaseBuilder()
+        .setName(toDbName)
+        .create(client, conf);
+
+    Table table = new TableBuilder()
+        .setTableName(tableName)
+        .addCol("a", "int")
+        .addPartCol("p", "string")
+        .create(client, conf);
+
+    new PartitionBuilder()
+        .inTable(table)
+        .addValue(partVal)
+        .addToTable(client, conf);
+
+    schemaTool.moveTable(DEFAULT_CATALOG_NAME, DEFAULT_CATALOG_NAME, DEFAULT_DATABASE_NAME, toDbName, tableName);
+
+    Table fetchedTable = client.getTable(DEFAULT_CATALOG_NAME, toDbName, tableName);
+    Assert.assertNotNull(fetchedTable);
+    Assert.assertEquals(DEFAULT_CATALOG_NAME, fetchedTable.getCatName());
+    Assert.assertEquals(toDbName.toLowerCase(), fetchedTable.getDbName());
+
+    Partition fetchedPart =
+        client.getPartition(DEFAULT_CATALOG_NAME, toDbName, tableName, Collections.singletonList(partVal));
+    Assert.assertNotNull(fetchedPart);
+    Assert.assertEquals(DEFAULT_CATALOG_NAME, fetchedPart.getCatName());
+    Assert.assertEquals(toDbName.toLowerCase(), fetchedPart.getDbName());
+    Assert.assertEquals(tableName.toLowerCase(), fetchedPart.getTableName());
+  }
+
+  @Test
+  public void moveTableWithExistingTableOfSameNameAlreadyInTargetDatabase()
+      throws TException, HiveMetaException {
+    String toDbName = "clobberTableDb";
+    String tableName = "clobberTableTable";
+
+    Database toDb = new DatabaseBuilder()
+        .setName(toDbName)
+        .create(client, conf);
+
+    new TableBuilder()
+        .setTableName(tableName)
+        .addCol("a", "int")
+        .create(client, conf);
+
+    new TableBuilder()
+        .inDb(toDb)
+        .setTableName(tableName)
+        .addCol("b", "varchar(32)")
+        .create(client, conf);
+
+    try {
+      schemaTool.moveTable(DEFAULT_CATALOG_NAME, DEFAULT_CATALOG_NAME, DEFAULT_DATABASE_NAME,
+          toDbName, tableName);
+      Assert.fail("Attempt to move table should have failed.");
+    } catch (HiveMetaException e) {
+      // good
+    }
+
+    // Make sure nothing really moved
+    Set<String> tableNames = new HashSet<>(client.getAllTables(DEFAULT_CATALOG_NAME, DEFAULT_DATABASE_NAME));
+    Assert.assertTrue(tableNames.contains(tableName.toLowerCase()));
+
+    // Make sure the table in the target database didn't get clobbered
+    Table fetchedTable =  client.getTable(DEFAULT_CATALOG_NAME, toDbName, tableName);
+    Assert.assertEquals("b", fetchedTable.getSd().getCols().get(0).getName());
+  }
+
+  @Test(expected = HiveMetaException.class)
+  public void moveNonExistentTable() throws TException, HiveMetaException {
+    String toDbName = "moveNonExistentTable";
+    new DatabaseBuilder()
+        .setName(toDbName)
+        .create(client, conf);
+    schemaTool.moveTable(DEFAULT_CATALOG_NAME, DEFAULT_CATALOG_NAME, DEFAULT_DATABASE_NAME, toDbName,
+        "nosuch");
+  }
+
+  @Test
+  public void moveTableToNonExistentDb() throws TException, HiveMetaException {
+    String tableName = "doomedToWander";
+    new TableBuilder()
+        .setTableName(tableName)
+        .addCol("a", "int")
+        .create(client, conf);
+
+    try {
+      schemaTool.moveTable(DEFAULT_CATALOG_NAME, DEFAULT_CATALOG_NAME, DEFAULT_DATABASE_NAME,
+          "nosuch", tableName);
+      Assert.fail("Attempt to move table to non-existent table should have failed.");
+    } catch (HiveMetaException e) {
+      // good
+    }
+
+    // Make sure nothing really moved
+    Set<String> tableNames = new HashSet<>(client.getAllTables(DEFAULT_CATALOG_NAME, DEFAULT_DATABASE_NAME));
+    Assert.assertTrue(tableNames.contains(tableName.toLowerCase()));
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/5bb3df0e/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java
index 3844c6f..61c565c 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java
@@ -9121,6 +9121,7 @@ public class ObjectStore implements RawStore, Configurable {
     Query query = null;
     try {
       openTransaction();
+      catName = normalizeIdentifier(catName);
       db = normalizeIdentifier(db);
       function = normalizeIdentifier(function);
       query = pm.newQuery(MFunction.class,


[37/50] [abbrv] hive git commit: HIVE-19374: Parse and process ALTER TABLE SET OWNER command syntax (Sergio Pena, reviewed by Vihang Karajgaonkar)

Posted by vg...@apache.org.
HIVE-19374: Parse and process ALTER TABLE SET OWNER command syntax (Sergio Pena, reviewed by Vihang Karajgaonkar)


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

Branch: refs/heads/branch-3.0.0
Commit: 14d0690b6af32c1aed8085590f49af941a64bc1c
Parents: ba7155d
Author: Sergio Pena <se...@cloudera.com>
Authored: Wed May 9 22:37:09 2018 -0700
Committer: Vineet Garg <vg...@apache.org>
Committed: Mon May 14 11:50:50 2018 -0700

----------------------------------------------------------------------
 .../ql/metadata/TestAlterTableMetadata.java     | 72 ++++++++++++++++++++
 .../org/apache/hadoop/hive/ql/exec/DDLTask.java |  5 ++
 .../apache/hadoop/hive/ql/metadata/Table.java   | 15 ++++
 .../formatting/JsonMetaDataFormatter.java       |  1 +
 .../formatting/MetaDataFormatUtils.java         |  1 +
 .../hive/ql/parse/DDLSemanticAnalyzer.java      | 17 +++++
 .../apache/hadoop/hive/ql/parse/HiveParser.g    |  8 +++
 .../hive/ql/parse/SemanticAnalyzerFactory.java  |  2 +
 .../hadoop/hive/ql/plan/AlterTableDesc.java     | 21 +++++-
 .../hadoop/hive/ql/plan/HiveOperation.java      |  1 +
 .../authorization/plugin/HiveOperationType.java |  1 +
 .../plugin/sqlstd/Operation2Privilege.java      |  2 +
 12 files changed, 145 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/14d0690b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/metadata/TestAlterTableMetadata.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/metadata/TestAlterTableMetadata.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/metadata/TestAlterTableMetadata.java
new file mode 100644
index 0000000..940a1d3
--- /dev/null
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/metadata/TestAlterTableMetadata.java
@@ -0,0 +1,72 @@
+/*
+ * 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.metadata;
+
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.api.PrincipalType;
+import org.apache.hadoop.hive.ql.DriverFactory;
+import org.apache.hadoop.hive.ql.IDriver;
+import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse;
+import org.apache.hadoop.hive.ql.session.SessionState;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestAlterTableMetadata {
+  @Test
+  public void testAlterTableOwner() throws HiveException {
+    /*
+     * This test verifies that the ALTER TABLE ... SET OWNER command will change the
+     * owner metadata of the table in HMS.
+     */
+
+    HiveConf conf = new HiveConf(this.getClass());
+    conf.set(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY.varname, "false");
+    SessionState.start(conf);
+    IDriver driver = DriverFactory.newDriver(conf);
+    CommandProcessorResponse resp;
+    Table table;
+
+    resp = driver.run("create table t1(id int)");
+    assertEquals(0, resp.getResponseCode());
+
+    // Changes the owner to a user and verify the change
+    resp = driver.run("alter table t1 set owner user u1");
+    assertEquals(0, resp.getResponseCode());
+
+    table = Hive.get(conf).getTable("t1");
+    assertEquals(PrincipalType.USER, table.getOwnerType());
+    assertEquals("u1", table.getOwner());
+
+    // Changes the owner to a group and verify the change
+    resp = driver.run("alter table t1 set owner group g1");
+    assertEquals(0, resp.getResponseCode());
+
+    table = Hive.get(conf).getTable("t1");
+    assertEquals(PrincipalType.GROUP, table.getOwnerType());
+    assertEquals("g1", table.getOwner());
+
+    // Changes the owner to a role and verify the change
+    resp = driver.run("alter table t1 set owner role r1");
+    assertEquals(0, resp.getResponseCode());
+
+    table = Hive.get(conf).getTable("t1");
+    assertEquals(PrincipalType.ROLE, table.getOwnerType());
+    assertEquals("r1", table.getOwner());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/14d0690b/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
index c9c5054..ab758c4 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
@@ -4326,6 +4326,11 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
       }
 
       tbl.setStoredAsSubDirectories(alterTbl.isStoredAsSubDirectories());
+    } else if (alterTbl.getOp() == AlterTableDesc.AlterTableTypes.OWNER) {
+      if (alterTbl.getOwnerPrincipal() != null) {
+        tbl.setOwner(alterTbl.getOwnerPrincipal().getName());
+        tbl.setOwnerType(alterTbl.getOwnerPrincipal().getType());
+      }
     } else if (alterTbl.getOp() == AlterTableDesc.AlterTableTypes.ALTERSKEWEDLOCATION) {
       // process location one-by-one
       Map<List<String>,String> locMaps = alterTbl.getSkewedLocations();

http://git-wip-us.apache.org/repos/asf/hive/blob/14d0690b/ql/src/java/org/apache/hadoop/hive/ql/metadata/Table.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/Table.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/Table.java
index 879b422..ba16f84 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/Table.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/Table.java
@@ -46,6 +46,7 @@ import org.apache.hadoop.hive.metastore.api.CreationMetadata;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.metastore.api.Order;
+import org.apache.hadoop.hive.metastore.api.PrincipalType;
 import org.apache.hadoop.hive.metastore.api.SerDeInfo;
 import org.apache.hadoop.hive.metastore.api.SkewedInfo;
 import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
@@ -744,6 +745,12 @@ public class Table implements Serializable {
   }
 
   /**
+   * @return The owner type of the table.
+   * @see org.apache.hadoop.hive.metastore.api.Table#getOwnerType()
+   */
+  public PrincipalType getOwnerType() { return tTable.getOwnerType(); }
+
+  /**
    * @return The table parameters.
    * @see org.apache.hadoop.hive.metastore.api.Table#getParameters()
    */
@@ -768,6 +775,14 @@ public class Table implements Serializable {
   }
 
   /**
+   * @param ownerType
+   * @see org.apache.hadoop.hive.metastore.api.Table#setOwnerType(org.apache.hadoop.hive.metastore.api.PrincipalType)
+   */
+  public void setOwnerType(PrincipalType ownerType) {
+    tTable.setOwnerType(ownerType);
+  }
+
+  /**
    * @param retention
    * @see org.apache.hadoop.hive.metastore.api.Table#setRetention(int)
    */

http://git-wip-us.apache.org/repos/asf/hive/blob/14d0690b/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/JsonMetaDataFormatter.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/JsonMetaDataFormatter.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/JsonMetaDataFormatter.java
index c21967c..e7b5af6 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/JsonMetaDataFormatter.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/JsonMetaDataFormatter.java
@@ -218,6 +218,7 @@ public class JsonMetaDataFormatter implements MetaDataFormatter {
     MapBuilder builder = MapBuilder.create();
 
     builder.put("tableName", tbl.getTableName());
+    builder.put("ownerType", (tbl.getOwnerType() != null) ? tbl.getOwnerType().name() : "null");
     builder.put("owner", tbl.getOwner());
     builder.put("location", tblLoc);
     builder.put("inputFormat", inputFormattCls);

http://git-wip-us.apache.org/repos/asf/hive/blob/14d0690b/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/MetaDataFormatUtils.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/MetaDataFormatUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/MetaDataFormatUtils.java
index af283e6..44687ef 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/MetaDataFormatUtils.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/MetaDataFormatUtils.java
@@ -425,6 +425,7 @@ public final class MetaDataFormatUtils {
   private static void getTableMetaDataInformation(StringBuilder tableInfo, Table  tbl,
       boolean isOutputPadded) {
     formatOutput("Database:", tbl.getDbName(), tableInfo);
+    formatOutput("OwnerType:", (tbl.getOwnerType() != null) ? tbl.getOwnerType().name() : "null", tableInfo);
     formatOutput("Owner:", tbl.getOwner(), tableInfo);
     formatOutput("CreateTime:", formatDate(tbl.getTTable().getCreateTime()), tableInfo);
     formatOutput("LastAccessTime:", formatDate(tbl.getTTable().getLastAccessTime()), tableInfo);

http://git-wip-us.apache.org/repos/asf/hive/blob/14d0690b/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
index f0b9eda..37e9845 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
@@ -365,6 +365,8 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
           analyzeAlterTableAddConstraint(ast, tableName);
       } else if(ast.getToken().getType() == HiveParser.TOK_ALTERTABLE_UPDATECOLUMNS) {
         analyzeAlterTableUpdateColumns(ast, tableName, partSpec);
+      } else if (ast.getToken().getType() == HiveParser.TOK_ALTERTABLE_OWNER) {
+        analyzeAlterTableOwner(ast, tableName);
       }
       break;
     }
@@ -1928,6 +1930,21 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
     }
   }
 
+  private void analyzeAlterTableOwner(ASTNode ast, String tableName) throws SemanticException {
+    PrincipalDesc ownerPrincipal = AuthorizationParseUtils.getPrincipalDesc((ASTNode) ast.getChild(0));
+
+    if (ownerPrincipal.getType() == null) {
+      throw new SemanticException("Owner type can't be null in alter table set owner command");
+    }
+
+    if (ownerPrincipal.getName() == null) {
+      throw new SemanticException("Owner name can't be null in alter table set owner command");
+    }
+
+    AlterTableDesc alterTblDesc  = new AlterTableDesc(tableName, ownerPrincipal);
+    rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(), alterTblDesc), conf));
+  }
+
   private void analyzeAlterTableLocation(ASTNode ast, String tableName,
       HashMap<String, String> partSpec) throws SemanticException {
 

http://git-wip-us.apache.org/repos/asf/hive/blob/14d0690b/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g b/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g
index 3712a53..8726974 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g
@@ -188,6 +188,7 @@ TOK_ALTERTABLE_COMPACT;
 TOK_ALTERTABLE_DROPCONSTRAINT;
 TOK_ALTERTABLE_ADDCONSTRAINT;
 TOK_ALTERTABLE_UPDATECOLUMNS;
+TOK_ALTERTABLE_OWNER;
 TOK_MSCK;
 TOK_SHOWDATABASES;
 TOK_SHOWTABLES;
@@ -1136,6 +1137,7 @@ alterTableStatementSuffix
     | alterStatementSuffixDropConstraint
     | alterStatementSuffixAddConstraint
     | partitionSpec? alterTblPartitionStatementSuffix -> alterTblPartitionStatementSuffix partitionSpec?
+    | alterStatementSuffixSetOwner
     ;
 
 alterTblPartitionStatementSuffix
@@ -1481,6 +1483,12 @@ alterStatementSuffixCompact
     -> ^(TOK_ALTERTABLE_COMPACT $compactType blocking? tableProperties?)
     ;
 
+alterStatementSuffixSetOwner
+@init { pushMsg("alter table set owner", state); }
+@after { popMsg(state); }
+    : KW_SET KW_OWNER principalName
+    -> ^(TOK_ALTERTABLE_OWNER principalName)
+    ;
 
 fileFormat
 @init { pushMsg("file format specification", state); }

http://git-wip-us.apache.org/repos/asf/hive/blob/14d0690b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzerFactory.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzerFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzerFactory.java
index 8200463..0f42ab8 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzerFactory.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzerFactory.java
@@ -94,6 +94,7 @@ public final class SemanticAnalyzerFactory {
     commandType.put(HiveParser.TOK_ALTERVIEW_DROPPROPERTIES, HiveOperation.ALTERVIEW_PROPERTIES);
     commandType.put(HiveParser.TOK_ALTERVIEW_ADDPARTS, HiveOperation.ALTERTABLE_ADDPARTS);
     commandType.put(HiveParser.TOK_ALTERVIEW_DROPPARTS, HiveOperation.ALTERTABLE_DROPPARTS);
+    commandType.put(HiveParser.TOK_ALTERTABLE_OWNER, HiveOperation.ALTERTABLE_OWNER);
     commandType.put(HiveParser.TOK_ALTERVIEW_RENAME, HiveOperation.ALTERVIEW_RENAME);
     commandType.put(HiveParser.TOK_ALTERVIEW, HiveOperation.ALTERVIEW_AS);
     commandType.put(HiveParser.TOK_ALTER_MATERIALIZED_VIEW_REWRITE,
@@ -244,6 +245,7 @@ public final class SemanticAnalyzerFactory {
           case HiveParser.TOK_ALTERTABLE_DROPPROPERTIES:
           case HiveParser.TOK_ALTERTABLE_EXCHANGEPARTITION:
           case HiveParser.TOK_ALTERTABLE_SKEWED:
+          case HiveParser.TOK_ALTERTABLE_OWNER:
           case HiveParser.TOK_ALTERTABLE_DROPCONSTRAINT:
           case HiveParser.TOK_ALTERTABLE_ADDCONSTRAINT:
           case HiveParser.TOK_ALTERTABLE_UPDATECOLUMNS:

http://git-wip-us.apache.org/repos/asf/hive/blob/14d0690b/ql/src/java/org/apache/hadoop/hive/ql/plan/AlterTableDesc.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/AlterTableDesc.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/AlterTableDesc.java
index a767796..0b04c0c 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/AlterTableDesc.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/AlterTableDesc.java
@@ -66,7 +66,7 @@ public class AlterTableDesc extends DDLDesc implements Serializable, DDLDesc.DDL
     ALTERSKEWEDLOCATION("alter skew location"), ALTERBUCKETNUM("alter bucket number"),
     ALTERPARTITION("alter partition"), COMPACT("compact"),
     TRUNCATE("truncate"), MERGEFILES("merge files"), DROPCONSTRAINT("drop constraint"), ADDCONSTRAINT("add constraint"),
-    UPDATECOLUMNS("update columns");
+    UPDATECOLUMNS("update columns"), OWNER("set owner");
     ;
 
     private final String name;
@@ -138,6 +138,7 @@ public class AlterTableDesc extends DDLDesc implements Serializable, DDLDesc.DDL
   List<SQLCheckConstraint> checkConstraintsCols;
   ReplicationSpec replicationSpec;
   private Long writeId = null;
+  PrincipalDesc ownerPrincipal;
 
   public AlterTableDesc() {
   }
@@ -367,6 +368,24 @@ public class AlterTableDesc extends DDLDesc implements Serializable, DDLDesc.DDL
     op = AlterTableTypes.ADDCONSTRAINT;
   }
 
+  public AlterTableDesc(String tableName, PrincipalDesc ownerPrincipal) {
+    op  = AlterTableTypes.OWNER;
+    this.oldName = tableName;
+    this.ownerPrincipal = ownerPrincipal;
+  }
+
+  /**
+   * @param ownerPrincipal the owner principal of the table
+   */
+  public void setOwnerPrincipal(PrincipalDesc ownerPrincipal) {
+    this.ownerPrincipal = ownerPrincipal;
+  }
+
+  @Explain(displayName="owner")
+  public PrincipalDesc getOwnerPrincipal() {
+    return this.ownerPrincipal;
+  }
+
   @Explain(displayName = "new columns", explainLevels = { Level.USER, Level.DEFAULT, Level.EXTENDED })
   public List<String> getNewColsString() {
     return Utilities.getFieldSchemaString(getNewCols());

http://git-wip-us.apache.org/repos/asf/hive/blob/14d0690b/ql/src/java/org/apache/hadoop/hive/ql/plan/HiveOperation.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/HiveOperation.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/HiveOperation.java
index cd4c206..ba3330b 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/HiveOperation.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/HiveOperation.java
@@ -51,6 +51,7 @@ public enum HiveOperation {
   ALTERTABLE_ARCHIVE("ALTERTABLE_ARCHIVE", new Privilege[]{Privilege.ALTER_DATA}, null),
   ALTERTABLE_UNARCHIVE("ALTERTABLE_UNARCHIVE", new Privilege[]{Privilege.ALTER_DATA}, null),
   ALTERTABLE_PROPERTIES("ALTERTABLE_PROPERTIES", new Privilege[]{Privilege.ALTER_METADATA}, null),
+  ALTERTABLE_OWNER("ALTERTABLE_OWNER", null, null),
   ALTERTABLE_SERIALIZER("ALTERTABLE_SERIALIZER", new Privilege[]{Privilege.ALTER_METADATA}, null),
   ALTERPARTITION_SERIALIZER("ALTERPARTITION_SERIALIZER", new Privilege[]{Privilege.ALTER_METADATA}, null),
   ALTERTABLE_SERDEPROPERTIES("ALTERTABLE_SERDEPROPERTIES", new Privilege[]{Privilege.ALTER_METADATA}, null),

http://git-wip-us.apache.org/repos/asf/hive/blob/14d0690b/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveOperationType.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveOperationType.java b/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveOperationType.java
index be5c062..09c15f0 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveOperationType.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveOperationType.java
@@ -53,6 +53,7 @@ public enum HiveOperationType {
   ALTERTABLE_ARCHIVE,
   ALTERTABLE_UNARCHIVE,
   ALTERTABLE_PROPERTIES,
+  ALTERTABLE_OWNER,
   ALTERTABLE_SERIALIZER,
   ALTERTABLE_PARTCOLTYPE,
   ALTERTABLE_DROPCONSTRAINT,

http://git-wip-us.apache.org/repos/asf/hive/blob/14d0690b/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/Operation2Privilege.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/Operation2Privilege.java b/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/Operation2Privilege.java
index 771dfb7..6137983 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/Operation2Privilege.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/Operation2Privilege.java
@@ -196,6 +196,8 @@ public class Operation2Privilege {
 (OWNER_PRIV_AR, OWNER_PRIV_AR));
     op2Priv.put(HiveOperationType.ALTERTABLE_PROPERTIES, PrivRequirement.newIOPrivRequirement
 (OWNER_PRIV_AR, OWNER_PRIV_AR));
+    op2Priv.put(HiveOperationType.ALTERTABLE_OWNER, PrivRequirement.newIOPrivRequirement
+(OWNER_PRIV_AR, OWNER_PRIV_AR));
     op2Priv.put(HiveOperationType.ALTERTABLE_SERIALIZER, PrivRequirement.newIOPrivRequirement
 (OWNER_PRIV_AR, OWNER_PRIV_AR));
     op2Priv.put(HiveOperationType.ALTERTABLE_PARTCOLTYPE, PrivRequirement.newIOPrivRequirement


[06/50] [abbrv] hive git commit: HIVE-19446: QueryCache: Transaction lists needed for pending cache entries (Jason Dere, reviewed by GopalV)

Posted by vg...@apache.org.
HIVE-19446: QueryCache: Transaction lists needed for pending cache entries (Jason Dere, reviewed by GopalV)


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

Branch: refs/heads/branch-3.0.0
Commit: d5d8982097e98e69f08e5398e6d6cd6c5d216867
Parents: 71be5ac
Author: Jason Dere <jd...@hortonworks.com>
Authored: Tue May 8 11:52:01 2018 -0700
Committer: Jason Dere <jd...@hortonworks.com>
Committed: Tue May 8 11:52:52 2018 -0700

----------------------------------------------------------------------
 ql/src/java/org/apache/hadoop/hive/ql/Driver.java     |  9 ++++++---
 .../hive/ql/cache/results/QueryResultsCache.java      | 14 +++++++++++---
 2 files changed, 17 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/d5d89820/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/Driver.java b/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
index 3987b5f..08f9a67 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
@@ -1978,10 +1978,14 @@ public class Driver implements IDriver {
     if (cacheUsage != null) {
       if (cacheUsage.getStatus() == CacheUsage.CacheStatus.CAN_CACHE_QUERY_RESULTS &&
           plan.getFetchTask() != null) {
+        ValidTxnWriteIdList txnWriteIdList = null;
+        if (plan.hasAcidResourcesInQuery()) {
+          txnWriteIdList = AcidUtils.getValidTxnWriteIdList(conf);
+        }
         // The results of this query execution might be cacheable.
         // Add a placeholder entry in the cache so other queries know this result is pending.
         CacheEntry pendingCacheEntry =
-            QueryResultsCache.getInstance().addToCache(cacheUsage.getQueryInfo());
+            QueryResultsCache.getInstance().addToCache(cacheUsage.getQueryInfo(), txnWriteIdList);
         if (pendingCacheEntry != null) {
           // Update cacheUsage to reference the pending entry.
           this.cacheUsage.setCacheEntry(pendingCacheEntry);
@@ -2012,8 +2016,7 @@ public class Driver implements IDriver {
         }
         boolean savedToCache = QueryResultsCache.getInstance().setEntryValid(
             cacheUsage.getCacheEntry(),
-            plan.getFetchTask().getWork(),
-            txnWriteIdList);
+            plan.getFetchTask().getWork());
         LOG.info("savedToCache: {}", savedToCache);
         if (savedToCache) {
           useFetchFromCache(cacheUsage.getCacheEntry());

http://git-wip-us.apache.org/repos/asf/hive/blob/d5d89820/ql/src/java/org/apache/hadoop/hive/ql/cache/results/QueryResultsCache.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/cache/results/QueryResultsCache.java b/ql/src/java/org/apache/hadoop/hive/ql/cache/results/QueryResultsCache.java
index 6734ac5..11b9c27 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/cache/results/QueryResultsCache.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/cache/results/QueryResultsCache.java
@@ -488,11 +488,12 @@ public final class QueryResultsCache {
    * @param queryInfo
    * @return
    */
-  public CacheEntry addToCache(QueryInfo queryInfo) {
+  public CacheEntry addToCache(QueryInfo queryInfo, ValidTxnWriteIdList txnWriteIdList) {
     // Create placeholder entry with PENDING state.
     String queryText = queryInfo.getLookupInfo().getQueryText();
     CacheEntry addedEntry = new CacheEntry();
     addedEntry.queryInfo = queryInfo;
+    addedEntry.txnWriteIdList = txnWriteIdList;
 
     Lock writeLock = rwLock.writeLock();
     try {
@@ -522,7 +523,7 @@ public final class QueryResultsCache {
    * @param fetchWork
    * @return
    */
-  public boolean setEntryValid(CacheEntry cacheEntry, FetchWork fetchWork, ValidTxnWriteIdList txnWriteIdList) {
+  public boolean setEntryValid(CacheEntry cacheEntry, FetchWork fetchWork) {
     String queryText = cacheEntry.getQueryText();
     boolean dataDirMoved = false;
     Path queryResultsPath = null;
@@ -571,7 +572,6 @@ public final class QueryResultsCache {
         cacheEntry.cachedResultsPath = cachedResultsPath;
         cacheEntry.size = resultSize;
         this.cacheSize += resultSize;
-        cacheEntry.txnWriteIdList = txnWriteIdList;
 
         cacheEntry.setStatus(CacheEntryStatus.VALID);
         // Mark this entry as being in use. Caller will need to release later.
@@ -700,6 +700,14 @@ public final class QueryResultsCache {
           boolean writeIdCheckPassed = false;
           String tableName = tableUsed.getFullyQualifiedName();
           ValidTxnWriteIdList currentTxnWriteIdList = lookupInfo.txnWriteIdListProvider.get();
+          if (currentTxnWriteIdList == null) {
+            LOG.warn("Current query's txnWriteIdList is null!");
+            return false;
+          }
+          if (entry.txnWriteIdList == null) {
+            LOG.warn("Cache entry's txnWriteIdList is null!");
+            return false;
+          }
           ValidWriteIdList currentWriteIdForTable =
               currentTxnWriteIdList.getTableValidWriteIdList(tableName);
           ValidWriteIdList cachedWriteIdForTable = entry.txnWriteIdList.getTableValidWriteIdList(tableName);


[32/50] [abbrv] hive git commit: Addendum to HIVE-19384: Vectorization: IfExprTimestamp* do not handle NULLs correctly

Posted by vg...@apache.org.
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/685fc9f4
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/685fc9f4
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/685fc9f4

Branch: refs/heads/branch-3.0.0
Commit: 685fc9f4c1b7d3d49b262e2280fec63f3d52bcb9
Parents: e2d4f34
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:41 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/685fc9f4/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/685fc9f4/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/685fc9f4/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/685fc9f4/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/685fc9f4/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/685fc9f4/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/685fc9f4/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/685fc9f4/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/685fc9f4/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/685fc9f4/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/685fc9f4/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/685fc9f4/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;
-  }
-}


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

Posted by vg...@apache.org.
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/e2d4f347
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/e2d4f347
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/e2d4f347

Branch: refs/heads/branch-3.0.0
Commit: e2d4f3476f67331e3690903ad12a30aeb47126d0
Parents: e941bea
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:58:43 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 |  48 +-
 .../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, 2813 insertions(+), 411 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/e2d4f347/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/e2d4f347/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/e2d4f347/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/e2d4f347/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/e2d4f347/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/e2d4f347/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/e2d4f347/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/e2d4f347/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/e2d4f347/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/e2d4f347/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;
+      }
     }
   }
 


[18/50] [abbrv] hive git commit: HIVE-19347: TestTriggersWorkloadManager tests are failing consistently (Matt McCline, reviewed by Sergey Shelukhin)

Posted by vg...@apache.org.
HIVE-19347: TestTriggersWorkloadManager tests are failing consistently (Matt McCline, reviewed by Sergey Shelukhin)


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

Branch: refs/heads/branch-3.0.0
Commit: e40aa049ceb07ecea426234e88278b9ac3516978
Parents: c03ae0b
Author: Matt McCline <mm...@hortonworks.com>
Authored: Fri May 11 02:48:07 2018 -0500
Committer: Matt McCline <mm...@hortonworks.com>
Committed: Fri May 11 02:48:47 2018 -0500

----------------------------------------------------------------------
 .../hadoop/hive/ql/io/HiveInputFormat.java      | 27 ++++----------------
 1 file changed, 5 insertions(+), 22 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/e40aa049/ql/src/java/org/apache/hadoop/hive/ql/io/HiveInputFormat.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/HiveInputFormat.java b/ql/src/java/org/apache/hadoop/hive/ql/io/HiveInputFormat.java
index b25bb1d..6c6eeff 100755
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/HiveInputFormat.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/HiveInputFormat.java
@@ -284,7 +284,11 @@ public class HiveInputFormat<K extends WritableComparable, V extends Writable>
         }
         return inputFormat;
       }
-      serde = findSerDeForLlapSerDeIf(conf, part);
+      try {
+        serde = part.getDeserializer(conf);
+      } catch (Exception e) {
+        throw new HiveException("Error creating SerDe for LLAP IO", e);
+      }
     }
     if (isSupported && isVectorized) {
       InputFormat<?, ?> wrappedIf = llapIo.getInputFormat(inputFormat, serde);
@@ -319,27 +323,6 @@ public class HiveInputFormat<K extends WritableComparable, V extends Writable>
     return false;
   }
 
-  private static Deserializer findSerDeForLlapSerDeIf(
-      Configuration conf, PartitionDesc part) throws HiveException {
-    VectorPartitionDesc vpart =  part.getVectorPartitionDesc();
-    if (vpart != null) {
-      VectorMapOperatorReadType old = vpart.getVectorMapOperatorReadType();
-      if (old != VectorMapOperatorReadType.VECTORIZED_INPUT_FILE_FORMAT) {
-        if (LOG.isInfoEnabled()) {
-          LOG.info("Resetting VectorMapOperatorReadType from " + old + " for partition "
-            + part.getTableName() + " " + part.getPartSpec());
-        }
-        vpart.setVectorMapOperatorReadType(
-            VectorMapOperatorReadType.VECTORIZED_INPUT_FILE_FORMAT);
-      }
-    }
-    try {
-      return part.getDeserializer(conf);
-    } catch (Exception e) {
-      throw new HiveException("Error creating SerDe for LLAP IO", e);
-    }
-  }
-
   public static void injectLlapCaches(InputFormat<WritableComparable, Writable> inputFormat,
       LlapIo<VectorizedRowBatch> llapIo) {
     LOG.info("Injecting LLAP caches into " + inputFormat.getClass().getCanonicalName());


[09/50] [abbrv] hive git commit: HIVE-19173 : Add Storage Handler runtime information as part of DESCRIBE EXTENDED (Nishant Bangarwa via Ashutosh Chauhan)

Posted by vg...@apache.org.
HIVE-19173 : Add Storage Handler runtime information as part of DESCRIBE EXTENDED (Nishant Bangarwa via Ashutosh Chauhan)

Signed-off-by: Ashutosh Chauhan <ha...@apache.org>


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

Branch: refs/heads/branch-3.0.0
Commit: aa040c5bfcea2257b4aa89f39832a7d6198a43a0
Parents: b135724
Author: Nishant Bangarwa <ni...@gmail.com>
Authored: Fri Apr 13 09:43:00 2018 -0700
Committer: Vineet Garg <vg...@apache.org>
Committed: Tue May 8 16:05:54 2018 -0700

----------------------------------------------------------------------
 .../hadoop/hive/druid/DruidStorageHandler.java  |  70 +++++-
 .../hive/druid/DruidStorageHandlerInfo.java     |  72 ++++++
 .../hive/druid/json/KafkaSupervisorReport.java  | 231 +++++++++++++++++++
 .../hadoop/hive/druid/json/TaskReportData.java  | 125 ++++++++++
 .../org/apache/hadoop/hive/ql/exec/DDLTask.java |   7 +-
 .../apache/hadoop/hive/ql/metadata/Hive.java    |  62 +++--
 .../hive/ql/metadata/HiveStorageHandler.java    |  17 +-
 .../hive/ql/metadata/StorageHandlerInfo.java    |  38 +++
 .../formatting/JsonMetaDataFormatter.java       |   6 +-
 .../metadata/formatting/MetaDataFormatter.java  |   4 +-
 .../formatting/TextMetaDataFormatter.java       |  11 +-
 .../clientpositive/druidkafkamini_basic.q       |   6 +-
 .../druid/druidkafkamini_basic.q.out            |  26 ++-
 13 files changed, 633 insertions(+), 42 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/aa040c5b/druid-handler/src/java/org/apache/hadoop/hive/druid/DruidStorageHandler.java
----------------------------------------------------------------------
diff --git a/druid-handler/src/java/org/apache/hadoop/hive/druid/DruidStorageHandler.java b/druid-handler/src/java/org/apache/hadoop/hive/druid/DruidStorageHandler.java
index bc08bd8..3e707e3 100644
--- a/druid-handler/src/java/org/apache/hadoop/hive/druid/DruidStorageHandler.java
+++ b/druid-handler/src/java/org/apache/hadoop/hive/druid/DruidStorageHandler.java
@@ -35,6 +35,7 @@ import com.metamx.http.client.HttpClientInit;
 import com.metamx.http.client.Request;
 import com.metamx.http.client.response.StatusResponseHandler;
 import com.metamx.http.client.response.StatusResponseHolder;
+
 import io.druid.data.input.impl.DimensionSchema;
 import io.druid.data.input.impl.DimensionsSpec;
 import io.druid.data.input.impl.InputRowParser;
@@ -59,6 +60,7 @@ import io.druid.segment.loading.SegmentLoadingException;
 import io.druid.storage.hdfs.HdfsDataSegmentPusher;
 import io.druid.storage.hdfs.HdfsDataSegmentPusherConfig;
 import io.druid.timeline.DataSegment;
+
 import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
@@ -69,6 +71,7 @@ import org.apache.hadoop.hive.druid.io.DruidOutputFormat;
 import org.apache.hadoop.hive.druid.io.DruidQueryBasedInputFormat;
 import org.apache.hadoop.hive.druid.io.DruidRecordWriter;
 import org.apache.hadoop.hive.druid.json.KafkaSupervisorIOConfig;
+import org.apache.hadoop.hive.druid.json.KafkaSupervisorReport;
 import org.apache.hadoop.hive.druid.json.KafkaSupervisorSpec;
 import org.apache.hadoop.hive.druid.json.KafkaSupervisorTuningConfig;
 import org.apache.hadoop.hive.druid.security.KerberosHttpClient;
@@ -82,6 +85,7 @@ import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.metastore.api.Table;
 import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
 import org.apache.hadoop.hive.ql.metadata.HiveStorageHandler;
+import org.apache.hadoop.hive.ql.metadata.StorageHandlerInfo;
 import org.apache.hadoop.hive.ql.plan.TableDesc;
 import org.apache.hadoop.hive.ql.security.authorization.DefaultHiveAuthorizationProvider;
 import org.apache.hadoop.hive.ql.security.authorization.HiveAuthorizationProvider;
@@ -94,6 +98,7 @@ import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.OutputFormat;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hive.common.util.ShutdownHookManager;
+
 import org.jboss.netty.handler.codec.http.HttpMethod;
 import org.jboss.netty.handler.codec.http.HttpResponseStatus;
 import org.joda.time.DateTime;
@@ -116,6 +121,8 @@ import java.util.Properties;
 import java.util.Set;
 import java.util.stream.Collectors;
 
+import javax.annotation.Nullable;
+
 import static org.apache.hadoop.hive.druid.DruidStorageHandlerUtils.JSON_MAPPER;
 
 /**
@@ -454,7 +461,7 @@ public class DruidStorageHandler extends DefaultHiveMetaHook implements HiveStor
         console.printInfo("Druid Kafka Ingestion Reset successful.");
       } else {
         throw new IOException(String
-            .format("Unable to stop Kafka Ingestion Druid status [%d] full response [%s]",
+            .format("Unable to reset Kafka Ingestion Druid status [%d] full response [%s]",
                 response.getStatus().getCode(), response.getContent()));
       }
     } catch (Exception e) {
@@ -486,7 +493,7 @@ public class DruidStorageHandler extends DefaultHiveMetaHook implements HiveStor
 
   }
 
-  public KafkaSupervisorSpec fetchKafkaIngestionSpec(Table table) {
+  private KafkaSupervisorSpec fetchKafkaIngestionSpec(Table table) {
     // Stop Kafka Ingestion first
     final String overlordAddress = Preconditions.checkNotNull(HiveConf
             .getVar(getConf(), HiveConf.ConfVars.HIVE_DRUID_OVERLORD_DEFAULT_ADDRESS),
@@ -512,7 +519,7 @@ public class DruidStorageHandler extends DefaultHiveMetaHook implements HiveStor
         return null;
       } else {
         throw new IOException(String
-            .format("Unable to stop Kafka Ingestion Druid status [%d] full response [%s]",
+            .format("Unable to fetch Kafka Ingestion Spec from Druid status [%d] full response [%s]",
                 response.getStatus().getCode(), response.getContent()));
       }
     } catch (Exception e) {
@@ -521,6 +528,46 @@ public class DruidStorageHandler extends DefaultHiveMetaHook implements HiveStor
   }
 
   /**
+   * Fetches kafka supervisor status report from druid overlod.
+   * @param table
+   * @return kafka supervisor report or null when druid overlord is unreachable.
+   */
+  @Nullable
+  private KafkaSupervisorReport fetchKafkaSupervisorReport(Table table) {
+    final String overlordAddress = Preconditions.checkNotNull(HiveConf
+                    .getVar(getConf(), HiveConf.ConfVars.HIVE_DRUID_OVERLORD_DEFAULT_ADDRESS),
+            "Druid Overlord Address is null");
+    String dataSourceName = Preconditions
+            .checkNotNull(getTableProperty(table, Constants.DRUID_DATA_SOURCE),
+                    "Druid Datasource name is null");
+    try {
+      StatusResponseHolder response = RetryUtils.retry(() -> getHttpClient().go(new Request(HttpMethod.GET,
+                      new URL(String
+                              .format("http://%s/druid/indexer/v1/supervisor/%s/status", overlordAddress,
+                                      dataSourceName))),
+              new StatusResponseHandler(
+                      Charset.forName("UTF-8"))).get(),
+              input -> input instanceof IOException,
+              getMaxRetryCount());
+      if (response.getStatus().equals(HttpResponseStatus.OK)) {
+        return DruidStorageHandlerUtils.JSON_MAPPER
+                .readValue(response.getContent(), KafkaSupervisorReport.class);
+        // Druid Returns 400 Bad Request when not found.
+      } else if (response.getStatus().equals(HttpResponseStatus.NOT_FOUND) || response.getStatus().equals(HttpResponseStatus.BAD_REQUEST)) {
+        LOG.info("No Kafka Supervisor found for datasource[%s]", dataSourceName);
+        return null;
+      } else {
+        LOG.error("Unable to fetch Kafka Supervisor status [%d] full response [%s]",
+                        response.getStatus().getCode(), response.getContent());
+        return null;
+      }
+    } catch (Exception e) {
+      LOG.error("Exception while fetching kafka ingestion spec from druid", e);
+      return null;
+    }
+  }
+  
+  /**
    * Creates metadata moves then commit the Segment's metadata to Druid metadata store in one TxN
    *
    * @param table Hive table
@@ -995,6 +1042,7 @@ public class DruidStorageHandler extends DefaultHiveMetaHook implements HiveStor
       updateKafkaIngestion(table);
     }
   }
+
   private static <T> Boolean getBooleanProperty(Table table, String propertyName) {
     String val = getTableProperty(table, propertyName);
     if (val == null) {
@@ -1057,4 +1105,20 @@ public class DruidStorageHandler extends DefaultHiveMetaHook implements HiveStor
   private int getMaxRetryCount() {
     return HiveConf.getIntVar(getConf(), HiveConf.ConfVars.HIVE_DRUID_MAX_TRIES);
   }
+
+  @Override
+  public StorageHandlerInfo getStorageHandlerInfo(Table table) throws MetaException {
+    if(isKafkaStreamingTable(table)){
+        KafkaSupervisorReport kafkaSupervisorReport = fetchKafkaSupervisorReport(table);
+        if(kafkaSupervisorReport == null){
+          return DruidStorageHandlerInfo.UNREACHABLE;
+        }
+        return new DruidStorageHandlerInfo(kafkaSupervisorReport);
+    }
+    else
+      // TODO: Currently we do not expose any runtime info for non-streaming tables.
+      // In future extend this add more information regarding table status.
+      // e.g. Total size of segments in druid, loadstatus of table on historical nodes etc.
+      return null;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/aa040c5b/druid-handler/src/java/org/apache/hadoop/hive/druid/DruidStorageHandlerInfo.java
----------------------------------------------------------------------
diff --git a/druid-handler/src/java/org/apache/hadoop/hive/druid/DruidStorageHandlerInfo.java b/druid-handler/src/java/org/apache/hadoop/hive/druid/DruidStorageHandlerInfo.java
new file mode 100644
index 0000000..f0e1750
--- /dev/null
+++ b/druid-handler/src/java/org/apache/hadoop/hive/druid/DruidStorageHandlerInfo.java
@@ -0,0 +1,72 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.druid;
+
+import io.druid.java.util.common.StringUtils;
+
+import org.apache.hadoop.hive.druid.json.KafkaSupervisorReport;
+import org.apache.hadoop.hive.ql.metadata.StorageHandlerInfo;
+
+/**
+ * DruidStorageHandlerInfo provides a runtime information for DruidStorageHandler.
+ */
+@SuppressWarnings("serial")
+public class DruidStorageHandlerInfo implements StorageHandlerInfo {
+
+  public static final StorageHandlerInfo UNREACHABLE = new StorageHandlerInfo() {
+    @Override
+    public String formatAsText() {
+      return "Druid Overlord is Unreachable, Runtime Status : unknown";
+    }
+  };
+
+  private final KafkaSupervisorReport kafkaSupervisorReport;
+
+  DruidStorageHandlerInfo(KafkaSupervisorReport kafkaSupervisorReport) {
+    this.kafkaSupervisorReport = kafkaSupervisorReport;
+  }
+
+  @Override
+  public String formatAsText() {
+    StringBuilder sb = new StringBuilder();
+    sb.append("Druid Storage Handler Runtime Status for " + kafkaSupervisorReport.getId());
+    sb.append("\n");
+    sb.append("kafkaPartitions=" + kafkaSupervisorReport.getPayload().getPartitions());
+    sb.append("\n");
+    sb.append("activeTasks=" + kafkaSupervisorReport.getPayload().getActiveTasks());
+    sb.append("\n");
+    sb.append("publishingTasks=" + kafkaSupervisorReport.getPayload().getPublishingTasks());
+    if (kafkaSupervisorReport.getPayload().getLatestOffsets() != null) {
+      sb.append("\n");
+      sb.append("latestOffsets=" + kafkaSupervisorReport.getPayload().getLatestOffsets());
+    }
+    if (kafkaSupervisorReport.getPayload().getMinimumLag() != null) {
+      sb.append("\n");
+      sb.append("minimumLag=" + kafkaSupervisorReport.getPayload().getMinimumLag());
+    }
+    if (kafkaSupervisorReport.getPayload().getAggregateLag() != null) {
+      sb.append("\n");
+      sb.append("aggregateLag=" + kafkaSupervisorReport.getPayload().getAggregateLag());
+    }
+    if (kafkaSupervisorReport.getPayload().getOffsetsLastUpdated() != null) {
+      sb.append("\n");
+      sb.append("lastUpdateTime=" + kafkaSupervisorReport.getPayload().getOffsetsLastUpdated());
+    }
+    return sb.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/aa040c5b/druid-handler/src/java/org/apache/hadoop/hive/druid/json/KafkaSupervisorReport.java
----------------------------------------------------------------------
diff --git a/druid-handler/src/java/org/apache/hadoop/hive/druid/json/KafkaSupervisorReport.java b/druid-handler/src/java/org/apache/hadoop/hive/druid/json/KafkaSupervisorReport.java
new file mode 100644
index 0000000..5a6756e
--- /dev/null
+++ b/druid-handler/src/java/org/apache/hadoop/hive/druid/json/KafkaSupervisorReport.java
@@ -0,0 +1,231 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.druid.json;
+
+import io.druid.guice.annotations.Json;
+import io.druid.indexing.overlord.supervisor.SupervisorReport;
+import io.druid.java.util.common.IAE;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.common.collect.Lists;
+
+import org.joda.time.DateTime;
+
+import java.util.List;
+import java.util.Map;
+
+import javax.annotation.Nullable;
+
+/**
+ * This class is copied from druid source code
+ * in order to avoid adding additional dependencies on druid-indexing-service.
+ */
+public class KafkaSupervisorReport extends SupervisorReport
+{
+  public static class KafkaSupervisorReportPayload
+  {
+    private final String dataSource;
+    private final String topic;
+    private final Integer partitions;
+    private final Integer replicas;
+    private final Long durationSeconds;
+    private final List<TaskReportData> activeTasks;
+    private final List<TaskReportData> publishingTasks;
+    private final Map<Integer, Long> latestOffsets;
+    private final Map<Integer, Long> minimumLag;
+    private final Long aggregateLag;
+    private final DateTime offsetsLastUpdated;
+
+    @JsonCreator
+    public KafkaSupervisorReportPayload(
+        @JsonProperty("dataSource") String dataSource,
+        @JsonProperty("topic") String topic,
+        @JsonProperty("partitions") Integer partitions,
+        @JsonProperty("replicas") Integer replicas,
+        @JsonProperty("durationSeconds") Long durationSeconds,
+        @Nullable @JsonProperty("latestOffsets") Map<Integer, Long> latestOffsets,
+        @Nullable @JsonProperty("minimumLag") Map<Integer, Long> minimumLag,
+        @Nullable @JsonProperty("aggregateLag") Long aggregateLag,
+        @Nullable @JsonProperty("offsetsLastUpdated") DateTime offsetsLastUpdated
+    )
+    {
+      this.dataSource = dataSource;
+      this.topic = topic;
+      this.partitions = partitions;
+      this.replicas = replicas;
+      this.durationSeconds = durationSeconds;
+      this.activeTasks = Lists.newArrayList();
+      this.publishingTasks = Lists.newArrayList();
+      this.latestOffsets = latestOffsets;
+      this.minimumLag = minimumLag;
+      this.aggregateLag = aggregateLag;
+      this.offsetsLastUpdated = offsetsLastUpdated;
+    }
+
+    @JsonProperty
+    public String getDataSource()
+    {
+      return dataSource;
+    }
+
+    @JsonProperty
+    public String getTopic()
+    {
+      return topic;
+    }
+
+    @JsonProperty
+    public Integer getPartitions()
+    {
+      return partitions;
+    }
+
+    @JsonProperty
+    public Integer getReplicas()
+    {
+      return replicas;
+    }
+
+    @JsonProperty
+    public Long getDurationSeconds()
+    {
+      return durationSeconds;
+    }
+
+    @JsonProperty
+    public List<TaskReportData> getActiveTasks()
+    {
+      return activeTasks;
+    }
+
+    @JsonProperty
+    public List<TaskReportData> getPublishingTasks()
+    {
+      return publishingTasks;
+    }
+
+    @JsonProperty
+    @JsonInclude(JsonInclude.Include.NON_NULL)
+    public Map<Integer, Long> getLatestOffsets()
+    {
+      return latestOffsets;
+    }
+
+    @JsonProperty
+    @JsonInclude(JsonInclude.Include.NON_NULL)
+    public Map<Integer, Long> getMinimumLag()
+    {
+      return minimumLag;
+    }
+
+    @JsonProperty
+    @JsonInclude(JsonInclude.Include.NON_NULL)
+    public Long getAggregateLag()
+    {
+      return aggregateLag;
+    }
+
+    @JsonProperty
+    public DateTime getOffsetsLastUpdated()
+    {
+      return offsetsLastUpdated;
+    }
+
+    @Override
+    public String toString()
+    {
+      return "{" +
+             "dataSource='" + dataSource + '\'' +
+             ", topic='" + topic + '\'' +
+             ", partitions=" + partitions +
+             ", replicas=" + replicas +
+             ", durationSeconds=" + durationSeconds +
+             ", active=" + activeTasks +
+             ", publishing=" + publishingTasks +
+             (latestOffsets != null ? ", latestOffsets=" + latestOffsets : "") +
+             (minimumLag != null ? ", minimumLag=" + minimumLag : "") +
+             (aggregateLag != null ? ", aggregateLag=" + aggregateLag : "") +
+             (offsetsLastUpdated != null ? ", offsetsLastUpdated=" + offsetsLastUpdated : "") +
+             '}';
+    }
+  }
+
+  private final KafkaSupervisorReportPayload payload;
+
+  @JsonCreator
+  public KafkaSupervisorReport(@JsonProperty("id") String id,
+          @JsonProperty("generationTime")DateTime generationTime,
+          @JsonProperty("payload") KafkaSupervisorReportPayload payload){
+    super(id, generationTime);
+    this.payload = payload;
+  }
+
+  public KafkaSupervisorReport(
+      String dataSource,
+      DateTime generationTime,
+      String topic,
+      Integer partitions,
+      Integer replicas,
+      Long durationSeconds,
+      @Nullable Map<Integer, Long> latestOffsets,
+      @Nullable Map<Integer, Long> minimumLag,
+      @Nullable Long aggregateLag,
+      @Nullable DateTime offsetsLastUpdated
+  ) {
+    this(dataSource, generationTime, new KafkaSupervisorReportPayload(
+            dataSource,
+            topic,
+            partitions,
+            replicas,
+            durationSeconds,
+            latestOffsets,
+            minimumLag,
+            aggregateLag,
+            offsetsLastUpdated
+    ));
+  }
+
+  @Override
+  public KafkaSupervisorReportPayload getPayload()
+  {
+    return payload;
+  }
+
+  public void addTask(TaskReportData data)
+  {
+    if (data.getType().equals(TaskReportData.TaskType.ACTIVE)) {
+      payload.activeTasks.add(data);
+    } else if (data.getType().equals(TaskReportData.TaskType.PUBLISHING)) {
+      payload.publishingTasks.add(data);
+    } else {
+      throw new IAE("Unknown task type [%s]", data.getType().name());
+    }
+  }
+
+  @Override
+  public String toString()
+  {
+    return "{" +
+           "id='" + getId() + '\'' +
+           ", generationTime=" + getGenerationTime() +
+           ", payload=" + payload +
+           '}';
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/aa040c5b/druid-handler/src/java/org/apache/hadoop/hive/druid/json/TaskReportData.java
----------------------------------------------------------------------
diff --git a/druid-handler/src/java/org/apache/hadoop/hive/druid/json/TaskReportData.java b/druid-handler/src/java/org/apache/hadoop/hive/druid/json/TaskReportData.java
new file mode 100644
index 0000000..94a3f7f
--- /dev/null
+++ b/druid-handler/src/java/org/apache/hadoop/hive/druid/json/TaskReportData.java
@@ -0,0 +1,125 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.druid.json;
+
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+import org.joda.time.DateTime;
+
+import java.util.Map;
+
+import javax.annotation.Nullable;
+
+/**
+ * This class is copied from druid source code
+ * in order to avoid adding additional dependencies on druid-indexing-service.
+ */
+public class TaskReportData
+{
+  public enum TaskType
+  {
+    ACTIVE, PUBLISHING, UNKNOWN
+  }
+
+  private final String id;
+  private final Map<Integer, Long> startingOffsets;
+  private final DateTime startTime;
+  private final Long remainingSeconds;
+  private final TaskType type;
+  private final Map<Integer, Long> currentOffsets;
+  private final Map<Integer, Long> lag;
+
+  public TaskReportData(
+      String id,
+      @Nullable Map<Integer, Long> startingOffsets,
+      @Nullable Map<Integer, Long> currentOffsets,
+      DateTime startTime,
+      Long remainingSeconds,
+      TaskType type,
+      @Nullable Map<Integer, Long> lag
+  )
+  {
+    this.id = id;
+    this.startingOffsets = startingOffsets;
+    this.currentOffsets = currentOffsets;
+    this.startTime = startTime;
+    this.remainingSeconds = remainingSeconds;
+    this.type = type;
+    this.lag = lag;
+  }
+
+  @JsonProperty
+  public String getId()
+  {
+    return id;
+  }
+
+  @JsonProperty
+  @JsonInclude(JsonInclude.Include.NON_NULL)
+  public Map<Integer, Long> getStartingOffsets()
+  {
+    return startingOffsets;
+  }
+
+  @JsonProperty
+  @JsonInclude(JsonInclude.Include.NON_NULL)
+  public Map<Integer, Long> getCurrentOffsets()
+  {
+    return currentOffsets;
+  }
+
+  @JsonProperty
+  public DateTime getStartTime()
+  {
+    return startTime;
+  }
+
+  @JsonProperty
+  public Long getRemainingSeconds()
+  {
+    return remainingSeconds;
+  }
+
+  @JsonProperty
+  public TaskType getType()
+  {
+    return type;
+  }
+
+  @JsonProperty
+  @JsonInclude(JsonInclude.Include.NON_NULL)
+  public Map<Integer, Long> getLag()
+  {
+    return lag;
+  }
+
+  @Override
+  public String toString()
+  {
+    return "{" +
+           "id='" + id + '\'' +
+           (startingOffsets != null ? ", startingOffsets=" + startingOffsets : "") +
+           (currentOffsets != null ? ", currentOffsets=" + currentOffsets : "") +
+           ", startTime=" + startTime +
+           ", remainingSeconds=" + remainingSeconds +
+           (lag != null ? ", lag=" + lag : "") +
+           '}';
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/aa040c5b/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
index b3c95eb..c9c5054 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
@@ -19,7 +19,6 @@
 package org.apache.hadoop.hive.ql.exec;
 
 import static org.apache.commons.lang.StringUtils.join;
-import static org.apache.hadoop.hive.metastore.Warehouse.DEFAULT_CATALOG_NAME;
 import static org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.META_TABLE_STORAGE;
 
 import java.io.BufferedWriter;
@@ -78,7 +77,6 @@ import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.metastore.DefaultHiveMetaHook;
 import org.apache.hadoop.hive.metastore.HiveMetaHook;
 import org.apache.hadoop.hive.metastore.HiveMetaStoreUtils;
-import org.apache.hadoop.hive.metastore.IMetaStoreClient;
 import org.apache.hadoop.hive.metastore.PartitionDropOptions;
 import org.apache.hadoop.hive.metastore.StatObjectConverter;
 import org.apache.hadoop.hive.metastore.TableType;
@@ -166,6 +164,7 @@ import org.apache.hadoop.hive.ql.metadata.NotNullConstraint;
 import org.apache.hadoop.hive.ql.metadata.Partition;
 import org.apache.hadoop.hive.ql.metadata.PartitionIterable;
 import org.apache.hadoop.hive.ql.metadata.PrimaryKeyInfo;
+import org.apache.hadoop.hive.ql.metadata.StorageHandlerInfo;
 import org.apache.hadoop.hive.ql.metadata.Table;
 import org.apache.hadoop.hive.ql.metadata.UniqueConstraint;
 import org.apache.hadoop.hive.ql.metadata.formatting.MetaDataFormatUtils;
@@ -3757,6 +3756,7 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
       NotNullConstraint nnInfo = null;
       DefaultConstraint dInfo = null;
       CheckConstraint cInfo = null;
+      StorageHandlerInfo storageHandlerInfo = null;
       if (descTbl.isExt() || descTbl.isFormatted()) {
         pkInfo = db.getPrimaryKeys(tbl.getDbName(), tbl.getTableName());
         fkInfo = db.getForeignKeys(tbl.getDbName(), tbl.getTableName());
@@ -3764,6 +3764,7 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
         nnInfo = db.getNotNullConstraints(tbl.getDbName(), tbl.getTableName());
         dInfo = db.getDefaultConstraints(tbl.getDbName(), tbl.getTableName());
         cInfo = db.getCheckConstraints(tbl.getDbName(), tbl.getTableName());
+        storageHandlerInfo = db.getStorageHandlerInfo(tbl);
       }
       fixDecimalColumnTypeName(cols);
       // In case the query is served by HiveServer2, don't pad it with spaces,
@@ -3772,7 +3773,7 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
       formatter.describeTable(outStream, colPath, tableName, tbl, part,
           cols, descTbl.isFormatted(), descTbl.isExt(),
           isOutputPadded, colStats,
-          pkInfo, fkInfo, ukInfo, nnInfo, dInfo, cInfo);
+          pkInfo, fkInfo, ukInfo, nnInfo, dInfo, cInfo, storageHandlerInfo);
 
       LOG.debug("DDLTask: written data for {}", tableName);
 

http://git-wip-us.apache.org/repos/asf/hive/blob/aa040c5b/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
index 3218f96..64b3f83 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
@@ -58,6 +58,7 @@ import java.util.concurrent.atomic.AtomicInteger;
 import java.util.Set;
 import java.util.stream.Collectors;
 
+import javax.annotation.Nullable;
 import javax.jdo.JDODataStoreException;
 
 import com.google.common.collect.ImmutableList;
@@ -95,7 +96,6 @@ import org.apache.hadoop.hive.common.ValidWriteIdList;
 import org.apache.hadoop.hive.common.classification.InterfaceAudience.LimitedPrivate;
 import org.apache.hadoop.hive.common.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.hive.common.log.InPlaceUpdate;
-import org.apache.hadoop.hive.conf.Constants;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.io.HdfsUtils;
@@ -4166,29 +4166,14 @@ private void constructOneLBLocationMap(FileStatus fSta,
   private IMetaStoreClient createMetaStoreClient(boolean allowEmbedded) throws MetaException {
 
     HiveMetaHookLoader hookLoader = new HiveMetaHookLoader() {
-        @Override
-        public HiveMetaHook getHook(
-          org.apache.hadoop.hive.metastore.api.Table tbl)
-          throws MetaException {
-
-          try {
-            if (tbl == null) {
-              return null;
-            }
-            HiveStorageHandler storageHandler =
-              HiveUtils.getStorageHandler(conf,
-                tbl.getParameters().get(META_TABLE_STORAGE));
-            if (storageHandler == null) {
-              return null;
-            }
-            return storageHandler.getMetaHook();
-          } catch (HiveException ex) {
-            LOG.error(StringUtils.stringifyException(ex));
-            throw new MetaException(
-              "Failed to load storage handler:  " + ex.getMessage());
-          }
-        }
-      };
+      @Override
+      public HiveMetaHook getHook(
+              org.apache.hadoop.hive.metastore.api.Table tbl)
+              throws MetaException {
+        HiveStorageHandler storageHandler = createStorageHandler(tbl);
+        return storageHandler == null ? null : storageHandler.getMetaHook();
+      }
+    };
 
     if (conf.getBoolVar(ConfVars.METASTORE_FASTPATH)) {
       return new SessionHiveMetaStoreClient(conf, hookLoader, allowEmbedded);
@@ -4198,6 +4183,22 @@ private void constructOneLBLocationMap(FileStatus fSta,
     }
   }
 
+  @Nullable
+  private HiveStorageHandler createStorageHandler(org.apache.hadoop.hive.metastore.api.Table tbl) throws MetaException {
+    try {
+      if (tbl == null) {
+        return null;
+      }
+      HiveStorageHandler storageHandler =
+              HiveUtils.getStorageHandler(conf, tbl.getParameters().get(META_TABLE_STORAGE));
+      return storageHandler;
+    } catch (HiveException ex) {
+      LOG.error(StringUtils.stringifyException(ex));
+      throw new MetaException(
+              "Failed to load storage handler:  " + ex.getMessage());
+    }
+  }
+
   public static class SchemaException extends MetaException {
     private static final long serialVersionUID = 1L;
     public SchemaException(String message) {
@@ -5115,4 +5116,15 @@ private void constructOneLBLocationMap(FileStatus fSta,
       throw new HiveException(e);
     }
   }
-};
+
+  @Nullable
+  public StorageHandlerInfo getStorageHandlerInfo(Table table)
+      throws HiveException {
+    try {
+      HiveStorageHandler storageHandler = createStorageHandler(table.getTTable());
+      return storageHandler == null ? null : storageHandler.getStorageHandlerInfo(table.getTTable());
+    } catch (Exception e) {
+      throw new HiveException(e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/aa040c5b/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveStorageHandler.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveStorageHandler.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveStorageHandler.java
index 99bb9f6..1696243 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveStorageHandler.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveStorageHandler.java
@@ -18,12 +18,15 @@
 
 package org.apache.hadoop.hive.ql.metadata;
 
+import java.util.Collections;
 import java.util.Map;
 
 import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.hive.common.classification.InterfaceAudience;
 import org.apache.hadoop.hive.common.classification.InterfaceStability;
 import org.apache.hadoop.hive.metastore.HiveMetaHook;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.Table;
 import org.apache.hadoop.hive.ql.plan.TableDesc;
 import org.apache.hadoop.hive.serde2.AbstractSerDe;
 import org.apache.hadoop.hive.ql.security.authorization.HiveAuthorizationProvider;
@@ -149,7 +152,19 @@ public interface HiveStorageHandler extends Configurable {
    * Called just before submitting MapReduce job.
    *
    * @param tableDesc descriptor for the table being accessed
-   * @param JobConf jobConf for MapReduce job
+   * @param jobConf jobConf for MapReduce job
    */
   public void configureJobConf(TableDesc tableDesc, JobConf jobConf);
+
+  /**
+   * Used to fetch runtime information about storage handler during DESCRIBE EXTENDED statement
+   *
+   * @param table table definition
+   * @return StorageHandlerInfo containing runtime information about storage handler
+   * OR `null` if the storage handler choose to not provide any runtime information.
+   */
+  public default StorageHandlerInfo getStorageHandlerInfo(Table table) throws MetaException
+  {
+    return null;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/aa040c5b/ql/src/java/org/apache/hadoop/hive/ql/metadata/StorageHandlerInfo.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/StorageHandlerInfo.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/StorageHandlerInfo.java
new file mode 100644
index 0000000..dbc44a6
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/StorageHandlerInfo.java
@@ -0,0 +1,38 @@
+/*
+ * 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.metadata;
+
+import org.apache.hadoop.hive.metastore.api.SQLForeignKey;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+
+/**
+ * StorageHandlerInfo is a marker interface used to provide runtime information associated with a storage handler.
+ */
+public interface StorageHandlerInfo extends Serializable {
+  /**
+   * Called from Describe Extended Statement when Formatter is Text Formatter.
+   * @return Formatted StorageHandlerInfo as String
+   */
+  String formatAsText();
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/aa040c5b/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/JsonMetaDataFormatter.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/JsonMetaDataFormatter.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/JsonMetaDataFormatter.java
index cd70eee..c21967c 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/JsonMetaDataFormatter.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/JsonMetaDataFormatter.java
@@ -51,6 +51,7 @@ import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.metadata.NotNullConstraint;
 import org.apache.hadoop.hive.ql.metadata.Partition;
 import org.apache.hadoop.hive.ql.metadata.PrimaryKeyInfo;
+import org.apache.hadoop.hive.ql.metadata.StorageHandlerInfo;
 import org.apache.hadoop.hive.ql.metadata.Table;
 import org.apache.hadoop.hive.ql.metadata.UniqueConstraint;
 import org.codehaus.jackson.JsonGenerator;
@@ -117,7 +118,7 @@ public class JsonMetaDataFormatter implements MetaDataFormatter {
       boolean isOutputPadded, List<ColumnStatisticsObj> colStats,
       PrimaryKeyInfo pkInfo, ForeignKeyInfo fkInfo,
       UniqueConstraint ukInfo, NotNullConstraint nnInfo, DefaultConstraint dInfo,
-                            CheckConstraint cInfo) throws HiveException {
+      CheckConstraint cInfo, StorageHandlerInfo storageHandlerInfo) throws HiveException {
     MapBuilder builder = MapBuilder.create();
     builder.put("columns", makeColsUnformatted(cols));
 
@@ -146,6 +147,9 @@ public class JsonMetaDataFormatter implements MetaDataFormatter {
       if (cInfo != null && !cInfo.getCheckConstraints().isEmpty()) {
         builder.put("checkConstraintInfo", cInfo);
       }
+      if(storageHandlerInfo != null) {
+        builder.put("storageHandlerInfo", storageHandlerInfo.toString());
+      }
     }
 
     asJson(out, builder.build());

http://git-wip-us.apache.org/repos/asf/hive/blob/aa040c5b/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/MetaDataFormatter.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/MetaDataFormatter.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/MetaDataFormatter.java
index ed2cdd1..d15016c 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/MetaDataFormatter.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/MetaDataFormatter.java
@@ -38,6 +38,7 @@ import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.metadata.NotNullConstraint;
 import org.apache.hadoop.hive.ql.metadata.Partition;
 import org.apache.hadoop.hive.ql.metadata.PrimaryKeyInfo;
+import org.apache.hadoop.hive.ql.metadata.StorageHandlerInfo;
 import org.apache.hadoop.hive.ql.metadata.Table;
 import org.apache.hadoop.hive.ql.metadata.UniqueConstraint;
 
@@ -91,7 +92,8 @@ public interface MetaDataFormatter {
       boolean isFormatted, boolean isExt,
       boolean isOutputPadded, List<ColumnStatisticsObj> colStats,
       PrimaryKeyInfo pkInfo, ForeignKeyInfo fkInfo,
-      UniqueConstraint ukInfo, NotNullConstraint nnInfo, DefaultConstraint dInfo, CheckConstraint cInfo)
+      UniqueConstraint ukInfo, NotNullConstraint nnInfo, DefaultConstraint dInfo, CheckConstraint cInfo,
+      StorageHandlerInfo storageHandlerInfo)
           throws HiveException;
 
   /**

http://git-wip-us.apache.org/repos/asf/hive/blob/aa040c5b/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/TextMetaDataFormatter.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/TextMetaDataFormatter.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/TextMetaDataFormatter.java
index 63a2969..2529923 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/TextMetaDataFormatter.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/TextMetaDataFormatter.java
@@ -29,6 +29,7 @@ import java.util.Map;
 import java.util.Set;
 
 import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
+import org.apache.hadoop.hive.ql.metadata.StorageHandlerInfo;
 import org.apache.hive.common.util.HiveStringUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -132,7 +133,8 @@ class TextMetaDataFormatter implements MetaDataFormatter {
       boolean isFormatted, boolean isExt,
       boolean isOutputPadded, List<ColumnStatisticsObj> colStats,
       PrimaryKeyInfo pkInfo, ForeignKeyInfo fkInfo,
-      UniqueConstraint ukInfo, NotNullConstraint nnInfo, DefaultConstraint dInfo, CheckConstraint cInfo)
+      UniqueConstraint ukInfo, NotNullConstraint nnInfo, DefaultConstraint dInfo, CheckConstraint cInfo,
+      StorageHandlerInfo storageHandlerInfo)
         throws HiveException {
     try {
       List<FieldSchema> partCols = tbl.isPartitioned() ? tbl.getPartCols() : null;
@@ -252,6 +254,13 @@ class TextMetaDataFormatter implements MetaDataFormatter {
               outStream.write(terminator);
             }
           }
+
+          if (storageHandlerInfo!= null) {
+            outStream.write(("StorageHandlerInfo").getBytes("UTF-8"));
+            outStream.write(terminator);
+            outStream.write(storageHandlerInfo.formatAsText().getBytes("UTF-8"));
+            outStream.write(terminator);
+          }
         }
       }
     } catch (IOException e) {

http://git-wip-us.apache.org/repos/asf/hive/blob/aa040c5b/ql/src/test/queries/clientpositive/druidkafkamini_basic.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/druidkafkamini_basic.q b/ql/src/test/queries/clientpositive/druidkafkamini_basic.q
index f4fd2a6..4c30cdd 100644
--- a/ql/src/test/queries/clientpositive/druidkafkamini_basic.q
+++ b/ql/src/test/queries/clientpositive/druidkafkamini_basic.q
@@ -9,7 +9,7 @@ CREATE TABLE druid_kafka_test(`__time` timestamp, page string, `user` string, la
         "druid.kafka.ingestion.useEarliestOffset" = "true",
         "druid.kafka.ingestion.maxRowsInMemory" = "5",
         "druid.kafka.ingestion.startDelay" = "PT1S",
-        "druid.kafka.ingestion.taskDuration" = "PT30S",
+        "druid.kafka.ingestion.taskDuration" = "PT20S",
         "druid.kafka.ingestion.period" = "PT1S"
         );
 
@@ -18,7 +18,7 @@ ALTER TABLE druid_kafka_test SET TBLPROPERTIES('druid.kafka.ingestion' = 'START'
 !curl -ss http://localhost:8081/druid/indexer/v1/supervisor;
 
 -- Sleep for some time for ingestion tasks to ingest events
-!sleep 50;
+!sleep 60;
 
 DESCRIBE druid_kafka_test;
 DESCRIBE EXTENDED druid_kafka_test;
@@ -32,7 +32,7 @@ Select page FROM druid_kafka_test order by page;
 ALTER TABLE druid_kafka_test SET TBLPROPERTIES('druid.kafka.ingestion' = 'RESET');
 
 -- Sleep for some time for ingestion tasks to ingest events
-!sleep 50;
+!sleep 60;
 
 DESCRIBE druid_kafka_test;
 DESCRIBE EXTENDED druid_kafka_test;

http://git-wip-us.apache.org/repos/asf/hive/blob/aa040c5b/ql/src/test/results/clientpositive/druid/druidkafkamini_basic.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/druid/druidkafkamini_basic.q.out b/ql/src/test/results/clientpositive/druid/druidkafkamini_basic.q.out
index 6f553fa..c2cc249 100644
--- a/ql/src/test/results/clientpositive/druid/druidkafkamini_basic.q.out
+++ b/ql/src/test/results/clientpositive/druid/druidkafkamini_basic.q.out
@@ -8,7 +8,7 @@ PREHOOK: query: CREATE TABLE druid_kafka_test(`__time` timestamp, page string, `
         "druid.kafka.ingestion.useEarliestOffset" = "true",
         "druid.kafka.ingestion.maxRowsInMemory" = "5",
         "druid.kafka.ingestion.startDelay" = "PT1S",
-        "druid.kafka.ingestion.taskDuration" = "PT30S",
+        "druid.kafka.ingestion.taskDuration" = "PT20S",
         "druid.kafka.ingestion.period" = "PT1S"
         )
 PREHOOK: type: CREATETABLE
@@ -24,7 +24,7 @@ POSTHOOK: query: CREATE TABLE druid_kafka_test(`__time` timestamp, page string,
         "druid.kafka.ingestion.useEarliestOffset" = "true",
         "druid.kafka.ingestion.maxRowsInMemory" = "5",
         "druid.kafka.ingestion.startDelay" = "PT1S",
-        "druid.kafka.ingestion.taskDuration" = "PT30S",
+        "druid.kafka.ingestion.taskDuration" = "PT20S",
         "druid.kafka.ingestion.period" = "PT1S"
         )
 POSTHOOK: type: CREATETABLE
@@ -65,6 +65,15 @@ added               	int                 	from deserializer
 deleted             	int                 	from deserializer   
 	 	 
 #### A masked pattern was here ####
+StorageHandlerInfo	 	 
+Druid Storage Handler Runtime Status for default.druid_kafka_test	 	 
+kafkaPartitions=1	 	 
+activeTasks=[]	 	 
+publishingTasks=[]	 	 
+latestOffsets={0=10}	 	 
+minimumLag={}	 	 
+aggregateLag=0	 	 
+#### A masked pattern was here ####
 PREHOOK: query: Select count(*) FROM druid_kafka_test
 PREHOOK: type: QUERY
 PREHOOK: Input: default@druid_kafka_test
@@ -126,6 +135,15 @@ added               	int                 	from deserializer
 deleted             	int                 	from deserializer   
 	 	 
 #### A masked pattern was here ####
+StorageHandlerInfo	 	 
+Druid Storage Handler Runtime Status for default.druid_kafka_test	 	 
+kafkaPartitions=1	 	 
+activeTasks=[]	 	 
+publishingTasks=[]	 	 
+latestOffsets={0=10}	 	 
+minimumLag={}	 	 
+aggregateLag=0	 	 
+#### A masked pattern was here ####
 PREHOOK: query: Select count(*) FROM druid_kafka_test
 PREHOOK: type: QUERY
 PREHOOK: Input: default@druid_kafka_test
@@ -331,7 +349,7 @@ STAGE PLANS:
                     druid.kafka.ingestion.maxRowsInMemory 5
                     druid.kafka.ingestion.period PT1S
                     druid.kafka.ingestion.startDelay PT1S
-                    druid.kafka.ingestion.taskDuration PT30S
+                    druid.kafka.ingestion.taskDuration PT20S
                     druid.kafka.ingestion.useEarliestOffset true
                     druid.query.granularity MINUTE
                     druid.query.json {"queryType":"scan","dataSource":"default.druid_kafka_test","intervals":["1900-01-01T00:00:00.000Z/3000-01-01T00:00:00.000Z"],"filter":{"type":"not","field":{"type":"selector","dimension":"language","value":null}},"columns":["language","user"],"resultFormat":"compactedList"}
@@ -370,7 +388,7 @@ STAGE PLANS:
                       druid.kafka.ingestion.maxRowsInMemory 5
                       druid.kafka.ingestion.period PT1S
                       druid.kafka.ingestion.startDelay PT1S
-                      druid.kafka.ingestion.taskDuration PT30S
+                      druid.kafka.ingestion.taskDuration PT20S
                       druid.kafka.ingestion.useEarliestOffset true
                       druid.query.granularity MINUTE
                       druid.query.json {"queryType":"scan","dataSource":"default.druid_kafka_test","intervals":["1900-01-01T00:00:00.000Z/3000-01-01T00:00:00.000Z"],"filter":{"type":"not","field":{"type":"selector","dimension":"language","value":null}},"columns":["language","user"],"resultFormat":"compactedList"}


[16/50] [abbrv] hive git commit: HIVE-19178: TestMiniTezCliDriver.testCliDriver[explainanalyze_5] failure (Jesus Camacho Rodriguez, reviewed by Vineet Garg)

Posted by vg...@apache.org.
HIVE-19178: TestMiniTezCliDriver.testCliDriver[explainanalyze_5] failure (Jesus Camacho Rodriguez, reviewed by Vineet Garg)


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

Branch: refs/heads/branch-3.0.0
Commit: d2c970022da1fe23c6e631979a6e1d5e17229e3f
Parents: 3c3880c
Author: Jesus Camacho Rodriguez <jc...@apache.org>
Authored: Tue May 8 16:53:55 2018 -0700
Committer: Jesus Camacho Rodriguez <jc...@apache.org>
Committed: Thu May 10 11:57:39 2018 -0700

----------------------------------------------------------------------
 .../hadoop/hive/ql/parse/ExplainSemanticAnalyzer.java     |  4 ++++
 .../results/clientpositive/tez/explainanalyze_5.q.out     | 10 +++++-----
 2 files changed, 9 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/d2c97002/ql/src/java/org/apache/hadoop/hive/ql/parse/ExplainSemanticAnalyzer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/ExplainSemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/ExplainSemanticAnalyzer.java
index 918cc5a..3a7d99d 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/ExplainSemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/ExplainSemanticAnalyzer.java
@@ -31,6 +31,8 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.common.FileUtils;
 import org.apache.hadoop.hive.common.StatsSetupConst;
+import org.apache.hadoop.hive.common.ValidTxnList;
+import org.apache.hadoop.hive.common.ValidTxnWriteIdList;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.ql.Context;
@@ -128,6 +130,8 @@ public class ExplainSemanticAnalyzer extends BaseSemanticAnalyzer {
       String query = ctx.getTokenRewriteStream().toString(input.getTokenStartIndex(),
           input.getTokenStopIndex());
       LOG.info("Explain analyze (running phase) for query " + query);
+      conf.unset(ValidTxnList.VALID_TXNS_KEY);
+      conf.unset(ValidTxnWriteIdList.VALID_TABLES_WRITEIDS_KEY);
       Context runCtx = null;
       try {
         runCtx = new Context(conf);

http://git-wip-us.apache.org/repos/asf/hive/blob/d2c97002/ql/src/test/results/clientpositive/tez/explainanalyze_5.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/explainanalyze_5.q.out b/ql/src/test/results/clientpositive/tez/explainanalyze_5.q.out
index e493211..8add8c0 100644
--- a/ql/src/test/results/clientpositive/tez/explainanalyze_5.q.out
+++ b/ql/src/test/results/clientpositive/tez/explainanalyze_5.q.out
@@ -34,19 +34,19 @@ Stage-2
         default@src_stats,src_stats,Tbl:COMPLETE,Col:COMPLETE
 
 PREHOOK: query: analyze table src_stats compute statistics for columns
-PREHOOK: type: QUERY
+PREHOOK: type: ANALYZE_TABLE
 PREHOOK: Input: default@src_stats
 PREHOOK: Output: default@src_stats
 PREHOOK: Output: hdfs://### HDFS PATH ###
 POSTHOOK: query: analyze table src_stats compute statistics for columns
-POSTHOOK: type: QUERY
+POSTHOOK: type: ANALYZE_TABLE
 POSTHOOK: Input: default@src_stats
 POSTHOOK: Output: default@src_stats
 POSTHOOK: Output: hdfs://### HDFS PATH ###
 PREHOOK: query: explain analyze analyze table src_stats compute statistics for columns
-PREHOOK: type: QUERY
+PREHOOK: type: ANALYZE_TABLE
 POSTHOOK: query: explain analyze analyze table src_stats compute statistics for columns
-POSTHOOK: type: QUERY
+POSTHOOK: type: ANALYZE_TABLE
 Vertex dependency in root stage
 Reducer 2 <- Map 1 (CUSTOM_SIMPLE_EDGE)
 
@@ -417,7 +417,7 @@ Stage-3
                       Output:["_col0"]
                       Filter Operator [FIL_8] (rows=1/8 width=3)
                         predicate:(cint < -1070551679)
-                        TableScan [TS_0] (rows=7484/12288 width=3)
+                        TableScan [TS_0] (rows=5865/12288 width=3)
                           default@acid_dot,acid_dot, ACID table,Tbl:COMPLETE,Col:NONE,Output:["cint"]
 
 PREHOOK: query: select count(*) from acid_dot


[39/50] [abbrv] hive git commit: HIVE-19108 : Vectorization and Parquet: Turning on vectorization in parquet_ppd_decimal.q causes Wrong Query Results (Haifeng Chen reviewed by Matt McCline and Vihang Karajgaonkar)

Posted by vg...@apache.org.
HIVE-19108 : Vectorization and Parquet: Turning on vectorization in parquet_ppd_decimal.q causes Wrong Query Results (Haifeng Chen reviewed by Matt McCline and Vihang Karajgaonkar)


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

Branch: refs/heads/branch-3.0.0
Commit: 6936c9c2a14c34665c4e8f20f8fc3b8b2af6afaa
Parents: d33f79f
Author: Vihang Karajgaonkar <vi...@cloudera.com>
Authored: Thu May 10 08:25:53 2018 -0700
Committer: Vihang Karajgaonkar <vi...@cloudera.com>
Committed: Mon May 14 12:54:59 2018 -0700

----------------------------------------------------------------------
 .../vector/expressions/CastDecimalToFloat.java  |  43 ++
 .../apache/hadoop/hive/ql/udf/UDFToFloat.java   |   4 +-
 .../vector/expressions/TestVectorTypeCasts.java |  48 ++
 .../clientpositive/parquet_ppd_decimal.q        |   1 -
 .../vectorization_parquet_ppd_decimal.q         | 169 ++++
 .../clientpositive/llap/vector_decimal_1.q.out  |   2 +-
 .../clientpositive/llap/vector_decimal_2.q.out  | 108 +--
 .../llap/vector_decimal_expressions.q.out       |   4 +-
 .../clientpositive/vector_decimal_1.q.out       |   2 +-
 .../vector_decimal_expressions.q.out            |   4 +-
 .../vectorization_parquet_ppd_decimal.q.out     | 766 +++++++++++++++++++
 11 files changed, 1088 insertions(+), 63 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/6936c9c2/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDecimalToFloat.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDecimalToFloat.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDecimalToFloat.java
new file mode 100644
index 0000000..4ef5422
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDecimalToFloat.java
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.ql.exec.vector.expressions;
+
+import org.apache.hadoop.hive.ql.exec.vector.DecimalColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.DoubleColumnVector;
+
+/**
+ * Cast a decimal to float based on decimal to double function.
+ *
+ */
+public class CastDecimalToFloat extends FuncDecimalToDouble {
+
+  private static final long serialVersionUID = 1L;
+
+  public CastDecimalToFloat() {
+    super();
+  }
+
+  public CastDecimalToFloat(int inputCol, int outputColumnNum) {
+    super(inputCol, outputColumnNum);
+  }
+
+  protected void func(DoubleColumnVector outV, DecimalColumnVector inV, int i) {
+    outV.vector[i] = inV.vector[i].floatValue();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/6936c9c2/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFToFloat.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFToFloat.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFToFloat.java
index fd49d1f..2872ff2 100755
--- a/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFToFloat.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFToFloat.java
@@ -20,7 +20,7 @@ package org.apache.hadoop.hive.ql.udf;
 
 import org.apache.hadoop.hive.ql.exec.UDF;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedExpressions;
-import org.apache.hadoop.hive.ql.exec.vector.expressions.CastDecimalToDouble;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.CastDecimalToFloat;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.CastStringToFloat;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.CastLongToFloatViaLongToDouble;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.CastTimestampToDouble;
@@ -42,7 +42,7 @@ import org.apache.hadoop.io.Text;
  *
  */
 @VectorizedExpressions({CastTimestampToDouble.class, CastLongToFloatViaLongToDouble.class,
-    CastDecimalToDouble.class, CastStringToFloat.class})
+    CastDecimalToFloat.class, CastStringToFloat.class})
 public class UDFToFloat extends UDF {
   private final FloatWritable floatWritable = new FloatWritable();
 

http://git-wip-us.apache.org/repos/asf/hive/blob/6936c9c2/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorTypeCasts.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorTypeCasts.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorTypeCasts.java
index 3f17d4c..8499da6 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorTypeCasts.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorTypeCasts.java
@@ -379,6 +379,54 @@ public class TestVectorTypeCasts {
   }
 
   @Test
+  public void testCastDecimalToFloat() throws HiveException {
+
+    final double eps = 0.00000000000001d; // tolerance to check float equality
+
+    double f1 = HiveDecimal.create("1.1").floatValue();
+    double f2 = HiveDecimal.create("-2.2").floatValue();
+    double f3 = HiveDecimal.create("9999999999999999.00").floatValue();
+
+    // test basic case
+    VectorizedRowBatch b = getBatchDecimalDouble();
+    VectorExpression expr = new CastDecimalToFloat(0, 1);
+    expr.evaluate(b);
+    DoubleColumnVector r = (DoubleColumnVector) b.cols[1];
+    assertEquals(f1, r.vector[0], eps);
+    assertEquals(f2, r.vector[1], eps);
+    assertEquals(f3, r.vector[2], eps);
+
+    // test with nulls in input
+    b = getBatchDecimalDouble();
+    b.cols[0].noNulls = false;
+    b.cols[0].isNull[1] = true;
+    expr.evaluate(b);
+    r = (DoubleColumnVector) b.cols[1];
+    assertFalse(r.noNulls);
+    assertTrue(r.isNull[1]);
+    assertFalse(r.isNull[0]);
+    assertEquals(f1, r.vector[0], eps);
+
+    // test repeating case
+    b = getBatchDecimalDouble();
+    b.cols[0].isRepeating = true;
+    expr.evaluate(b);
+    r = (DoubleColumnVector) b.cols[1];
+    assertTrue(r.isRepeating);
+    assertEquals(f1, r.vector[0], eps);
+
+    // test repeating nulls case
+    b = getBatchDecimalDouble();
+    b.cols[0].isRepeating = true;
+    b.cols[0].noNulls = false;
+    b.cols[0].isNull[0] = true;
+    expr.evaluate(b);
+    r = (DoubleColumnVector) b.cols[1];
+    assertTrue(r.isRepeating);
+    assertTrue(r.isNull[0]);
+  }
+
+  @Test
   public void testCastDecimalToString() throws HiveException {
     VectorizedRowBatch b = getBatchDecimalString();
     VectorExpression expr = new CastDecimalToString(0, 1);

http://git-wip-us.apache.org/repos/asf/hive/blob/6936c9c2/ql/src/test/queries/clientpositive/parquet_ppd_decimal.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/parquet_ppd_decimal.q b/ql/src/test/queries/clientpositive/parquet_ppd_decimal.q
index ade49a3..812732b 100644
--- a/ql/src/test/queries/clientpositive/parquet_ppd_decimal.q
+++ b/ql/src/test/queries/clientpositive/parquet_ppd_decimal.q
@@ -1,4 +1,3 @@
--- Suppress vectorization due to known bug.  See HIVE-19108.
 set hive.vectorized.execution.enabled=false;
 set hive.test.vectorized.execution.enabled.override=disable;
 

http://git-wip-us.apache.org/repos/asf/hive/blob/6936c9c2/ql/src/test/queries/clientpositive/vectorization_parquet_ppd_decimal.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/vectorization_parquet_ppd_decimal.q b/ql/src/test/queries/clientpositive/vectorization_parquet_ppd_decimal.q
new file mode 100644
index 0000000..4bc8f2f
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/vectorization_parquet_ppd_decimal.q
@@ -0,0 +1,169 @@
+--! qt:dataset:src1
+--! qt:dataset:src
+set hive.vectorized.execution.enabled=true;
+set hive.test.vectorized.execution.enabled.override=enable;
+
+SET hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat;
+SET mapred.min.split.size=1000;
+SET mapred.max.split.size=5000;
+set hive.llap.cache.allow.synthetic.fileid=true;
+
+create table newtypestbl(c char(10), v varchar(10), d decimal(5,3), da date) stored as parquet;
+
+insert overwrite table newtypestbl select * from (select cast("apple" as char(10)), cast("bee" as varchar(10)), 0.22, cast("1970-02-20" as date) from src src1 union all select cast("hello" as char(10)), cast("world" as varchar(10)), 11.22, cast("1970-02-27" as date) from src src2 limit 10) uniontbl;
+
+-- decimal data types (EQUAL, NOT_EQUAL, LESS_THAN, LESS_THAN_EQUALS, IN, BETWEEN tests)
+select * from newtypestbl where d=0.22;
+
+set hive.optimize.index.filter=true;
+select * from newtypestbl where d=0.22;
+
+set hive.optimize.index.filter=false;
+select * from newtypestbl where d='0.22';
+
+set hive.optimize.index.filter=true;
+select * from newtypestbl where d='0.22';
+
+set hive.optimize.index.filter=false;
+select * from newtypestbl where d=cast('0.22' as float);
+
+set hive.optimize.index.filter=true;
+select * from newtypestbl where d=cast('0.22' as float);
+
+set hive.optimize.index.filter=false;
+select * from newtypestbl where d!=0.22;
+
+set hive.optimize.index.filter=true;
+select * from newtypestbl where d!=0.22;
+
+set hive.optimize.index.filter=false;
+select * from newtypestbl where d!='0.22';
+
+set hive.optimize.index.filter=true;
+select * from newtypestbl where d!='0.22';
+
+set hive.optimize.index.filter=false;
+select * from newtypestbl where d!=cast('0.22' as float);
+
+set hive.optimize.index.filter=true;
+select * from newtypestbl where d!=cast('0.22' as float);
+
+set hive.optimize.index.filter=false;
+select * from newtypestbl where d<11.22;
+
+set hive.optimize.index.filter=true;
+select * from newtypestbl where d<11.22;
+
+set hive.optimize.index.filter=false;
+select * from newtypestbl where d<'11.22';
+
+set hive.optimize.index.filter=true;
+select * from newtypestbl where d<'11.22';
+
+set hive.optimize.index.filter=false;
+select * from newtypestbl where d<cast('11.22' as float);
+
+set hive.optimize.index.filter=true;
+select * from newtypestbl where d<cast('11.22' as float);
+
+set hive.optimize.index.filter=false;
+select * from newtypestbl where d<1;
+
+set hive.optimize.index.filter=true;
+select * from newtypestbl where d<1;
+
+set hive.optimize.index.filter=false;
+select * from newtypestbl where d<=11.22 sort by c;
+
+set hive.optimize.index.filter=true;
+select * from newtypestbl where d<=11.22 sort by c;
+
+set hive.optimize.index.filter=false;
+select * from newtypestbl where d<='11.22' sort by c;
+
+set hive.optimize.index.filter=true;
+select * from newtypestbl where d<='11.22' sort by c;
+
+set hive.optimize.index.filter=false;
+select * from newtypestbl where d<=cast('11.22' as float) sort by c;
+
+set hive.optimize.index.filter=true;
+select * from newtypestbl where d<=cast('11.22' as float) sort by c;
+
+set hive.optimize.index.filter=false;
+select * from newtypestbl where d<=cast('11.22' as decimal);
+
+set hive.optimize.index.filter=true;
+select * from newtypestbl where d<=cast('11.22' as decimal);
+
+set hive.optimize.index.filter=false;
+select * from newtypestbl where d<=11.22BD sort by c;
+
+set hive.optimize.index.filter=true;
+select * from newtypestbl where d<=11.22BD sort by c;
+
+set hive.optimize.index.filter=false;
+select * from newtypestbl where d<=12 sort by c;
+
+set hive.optimize.index.filter=true;
+select * from newtypestbl where d<=12 sort by c;
+
+set hive.optimize.index.filter=false;
+select * from newtypestbl where d in ('0.22', '1.0');
+
+set hive.optimize.index.filter=true;
+select * from newtypestbl where d in ('0.22', '1.0');
+
+set hive.optimize.index.filter=false;
+select * from newtypestbl where d in ('0.22', '11.22') sort by c;
+
+set hive.optimize.index.filter=true;
+select * from newtypestbl where d in ('0.22', '11.22') sort by c;
+
+set hive.optimize.index.filter=false;
+select * from newtypestbl where d in ('0.9', '1.0');
+
+set hive.optimize.index.filter=true;
+select * from newtypestbl where d in ('0.9', '1.0');
+
+set hive.optimize.index.filter=false;
+select * from newtypestbl where d in ('0.9', 0.22);
+
+set hive.optimize.index.filter=true;
+select * from newtypestbl where d in ('0.9', 0.22);
+
+set hive.optimize.index.filter=false;
+select * from newtypestbl where d in ('0.9', 0.22, cast('11.22' as float)) sort by c;
+
+set hive.optimize.index.filter=true;
+select * from newtypestbl where d in ('0.9', 0.22, cast('11.22' as float)) sort by c;
+
+set hive.optimize.index.filter=false;
+select * from newtypestbl where d between 0 and 1;
+
+set hive.optimize.index.filter=true;
+select * from newtypestbl where d between 0 and 1;
+
+set hive.optimize.index.filter=false;
+select * from newtypestbl where d between 0 and 1000 sort by c;
+
+set hive.optimize.index.filter=true;
+select * from newtypestbl where d between 0 and 1000 sort by c;
+
+set hive.optimize.index.filter=false;
+select * from newtypestbl where d between 0 and '2.0';
+
+set hive.optimize.index.filter=true;
+select * from newtypestbl where d between 0 and '2.0';
+
+set hive.optimize.index.filter=false;
+select * from newtypestbl where d between 0 and cast(3 as float);
+
+set hive.optimize.index.filter=true;
+select * from newtypestbl where d between 0 and cast(3 as float);
+
+set hive.optimize.index.filter=false;
+select * from newtypestbl where d between 1 and cast(30 as char(10));
+
+set hive.optimize.index.filter=true;
+select * from newtypestbl where d between 1 and cast(30 as char(10));

http://git-wip-us.apache.org/repos/asf/hive/blob/6936c9c2/ql/src/test/results/clientpositive/llap/vector_decimal_1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/vector_decimal_1.q.out b/ql/src/test/results/clientpositive/llap/vector_decimal_1.q.out
index 5107015..05c43fb 100644
--- a/ql/src/test/results/clientpositive/llap/vector_decimal_1.q.out
+++ b/ql/src/test/results/clientpositive/llap/vector_decimal_1.q.out
@@ -665,7 +665,7 @@ STAGE PLANS:
                         className: VectorSelectOperator
                         native: true
                         projectedOutputColumnNums: [4]
-                        selectExpressions: CastDecimalToDouble(col 0:decimal(4,2)) -> 4:float
+                        selectExpressions: CastDecimalToFloat(col 0:decimal(4,2)) -> 4:float
                     Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
                     Reduce Output Operator
                       key expressions: _col0 (type: float)

http://git-wip-us.apache.org/repos/asf/hive/blob/6936c9c2/ql/src/test/results/clientpositive/llap/vector_decimal_2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/vector_decimal_2.q.out b/ql/src/test/results/clientpositive/llap/vector_decimal_2.q.out
index bc596b3..bdb84bc 100644
--- a/ql/src/test/results/clientpositive/llap/vector_decimal_2.q.out
+++ b/ql/src/test/results/clientpositive/llap/vector_decimal_2.q.out
@@ -131,11 +131,11 @@ STAGE PLANS:
 PREHOOK: query: select cast(t as boolean) from decimal_2 order by t
 PREHOOK: type: QUERY
 PREHOOK: Input: default@decimal_2
-#### A masked pattern was here ####
+PREHOOK: Output: hdfs://### HDFS PATH ###
 POSTHOOK: query: select cast(t as boolean) from decimal_2 order by t
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@decimal_2
-#### A masked pattern was here ####
+POSTHOOK: Output: hdfs://### HDFS PATH ###
 true
 PREHOOK: query: explain vectorization detail
 select cast(t as tinyint) from decimal_2 order by t
@@ -247,11 +247,11 @@ STAGE PLANS:
 PREHOOK: query: select cast(t as tinyint) from decimal_2 order by t
 PREHOOK: type: QUERY
 PREHOOK: Input: default@decimal_2
-#### A masked pattern was here ####
+PREHOOK: Output: hdfs://### HDFS PATH ###
 POSTHOOK: query: select cast(t as tinyint) from decimal_2 order by t
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@decimal_2
-#### A masked pattern was here ####
+POSTHOOK: Output: hdfs://### HDFS PATH ###
 17
 PREHOOK: query: explain vectorization detail
 select cast(t as smallint) from decimal_2 order by t
@@ -363,11 +363,11 @@ STAGE PLANS:
 PREHOOK: query: select cast(t as smallint) from decimal_2 order by t
 PREHOOK: type: QUERY
 PREHOOK: Input: default@decimal_2
-#### A masked pattern was here ####
+PREHOOK: Output: hdfs://### HDFS PATH ###
 POSTHOOK: query: select cast(t as smallint) from decimal_2 order by t
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@decimal_2
-#### A masked pattern was here ####
+POSTHOOK: Output: hdfs://### HDFS PATH ###
 17
 PREHOOK: query: explain vectorization detail
 select cast(t as int) from decimal_2 order by t
@@ -479,11 +479,11 @@ STAGE PLANS:
 PREHOOK: query: select cast(t as int) from decimal_2 order by t
 PREHOOK: type: QUERY
 PREHOOK: Input: default@decimal_2
-#### A masked pattern was here ####
+PREHOOK: Output: hdfs://### HDFS PATH ###
 POSTHOOK: query: select cast(t as int) from decimal_2 order by t
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@decimal_2
-#### A masked pattern was here ####
+POSTHOOK: Output: hdfs://### HDFS PATH ###
 17
 PREHOOK: query: explain vectorization detail
 select cast(t as bigint) from decimal_2 order by t
@@ -595,11 +595,11 @@ STAGE PLANS:
 PREHOOK: query: select cast(t as bigint) from decimal_2 order by t
 PREHOOK: type: QUERY
 PREHOOK: Input: default@decimal_2
-#### A masked pattern was here ####
+PREHOOK: Output: hdfs://### HDFS PATH ###
 POSTHOOK: query: select cast(t as bigint) from decimal_2 order by t
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@decimal_2
-#### A masked pattern was here ####
+POSTHOOK: Output: hdfs://### HDFS PATH ###
 17
 PREHOOK: query: explain vectorization detail
 select cast(t as float) from decimal_2 order by t
@@ -638,7 +638,7 @@ STAGE PLANS:
                         className: VectorSelectOperator
                         native: true
                         projectedOutputColumnNums: [2]
-                        selectExpressions: CastDecimalToDouble(col 0:decimal(18,9)) -> 2:float
+                        selectExpressions: CastDecimalToFloat(col 0:decimal(18,9)) -> 2:float
                     Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
                     Reduce Output Operator
                       key expressions: _col0 (type: float)
@@ -711,11 +711,11 @@ STAGE PLANS:
 PREHOOK: query: select cast(t as float) from decimal_2 order by t
 PREHOOK: type: QUERY
 PREHOOK: Input: default@decimal_2
-#### A masked pattern was here ####
+PREHOOK: Output: hdfs://### HDFS PATH ###
 POSTHOOK: query: select cast(t as float) from decimal_2 order by t
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@decimal_2
-#### A masked pattern was here ####
+POSTHOOK: Output: hdfs://### HDFS PATH ###
 17.29
 PREHOOK: query: explain vectorization detail
 select cast(t as double) from decimal_2 order by t
@@ -827,11 +827,11 @@ STAGE PLANS:
 PREHOOK: query: select cast(t as double) from decimal_2 order by t
 PREHOOK: type: QUERY
 PREHOOK: Input: default@decimal_2
-#### A masked pattern was here ####
+PREHOOK: Output: hdfs://### HDFS PATH ###
 POSTHOOK: query: select cast(t as double) from decimal_2 order by t
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@decimal_2
-#### A masked pattern was here ####
+POSTHOOK: Output: hdfs://### HDFS PATH ###
 17.29
 PREHOOK: query: explain vectorization detail
 select cast(t as string) from decimal_2 order by t
@@ -943,11 +943,11 @@ STAGE PLANS:
 PREHOOK: query: select cast(t as string) from decimal_2 order by t
 PREHOOK: type: QUERY
 PREHOOK: Input: default@decimal_2
-#### A masked pattern was here ####
+PREHOOK: Output: hdfs://### HDFS PATH ###
 POSTHOOK: query: select cast(t as string) from decimal_2 order by t
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@decimal_2
-#### A masked pattern was here ####
+POSTHOOK: Output: hdfs://### HDFS PATH ###
 17.29
 PREHOOK: query: insert overwrite table decimal_2
   select cast('3404045.5044003' as decimal(18,9)) from src tablesample (1 rows)
@@ -1070,11 +1070,11 @@ STAGE PLANS:
 PREHOOK: query: select cast(t as boolean) from decimal_2 order by t
 PREHOOK: type: QUERY
 PREHOOK: Input: default@decimal_2
-#### A masked pattern was here ####
+PREHOOK: Output: hdfs://### HDFS PATH ###
 POSTHOOK: query: select cast(t as boolean) from decimal_2 order by t
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@decimal_2
-#### A masked pattern was here ####
+POSTHOOK: Output: hdfs://### HDFS PATH ###
 true
 PREHOOK: query: explain vectorization detail
 select cast(t as tinyint) from decimal_2 order by t
@@ -1186,11 +1186,11 @@ STAGE PLANS:
 PREHOOK: query: select cast(t as tinyint) from decimal_2 order by t
 PREHOOK: type: QUERY
 PREHOOK: Input: default@decimal_2
-#### A masked pattern was here ####
+PREHOOK: Output: hdfs://### HDFS PATH ###
 POSTHOOK: query: select cast(t as tinyint) from decimal_2 order by t
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@decimal_2
-#### A masked pattern was here ####
+POSTHOOK: Output: hdfs://### HDFS PATH ###
 NULL
 PREHOOK: query: explain vectorization detail
 select cast(t as smallint) from decimal_2 order by t
@@ -1302,11 +1302,11 @@ STAGE PLANS:
 PREHOOK: query: select cast(t as smallint) from decimal_2 order by t
 PREHOOK: type: QUERY
 PREHOOK: Input: default@decimal_2
-#### A masked pattern was here ####
+PREHOOK: Output: hdfs://### HDFS PATH ###
 POSTHOOK: query: select cast(t as smallint) from decimal_2 order by t
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@decimal_2
-#### A masked pattern was here ####
+POSTHOOK: Output: hdfs://### HDFS PATH ###
 NULL
 PREHOOK: query: explain vectorization detail
 select cast(t as int) from decimal_2 order by t
@@ -1418,11 +1418,11 @@ STAGE PLANS:
 PREHOOK: query: select cast(t as int) from decimal_2 order by t
 PREHOOK: type: QUERY
 PREHOOK: Input: default@decimal_2
-#### A masked pattern was here ####
+PREHOOK: Output: hdfs://### HDFS PATH ###
 POSTHOOK: query: select cast(t as int) from decimal_2 order by t
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@decimal_2
-#### A masked pattern was here ####
+POSTHOOK: Output: hdfs://### HDFS PATH ###
 3404045
 PREHOOK: query: explain vectorization detail
 select cast(t as bigint) from decimal_2 order by t
@@ -1534,11 +1534,11 @@ STAGE PLANS:
 PREHOOK: query: select cast(t as bigint) from decimal_2 order by t
 PREHOOK: type: QUERY
 PREHOOK: Input: default@decimal_2
-#### A masked pattern was here ####
+PREHOOK: Output: hdfs://### HDFS PATH ###
 POSTHOOK: query: select cast(t as bigint) from decimal_2 order by t
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@decimal_2
-#### A masked pattern was here ####
+POSTHOOK: Output: hdfs://### HDFS PATH ###
 3404045
 PREHOOK: query: explain vectorization detail
 select cast(t as float) from decimal_2 order by t
@@ -1577,7 +1577,7 @@ STAGE PLANS:
                         className: VectorSelectOperator
                         native: true
                         projectedOutputColumnNums: [2]
-                        selectExpressions: CastDecimalToDouble(col 0:decimal(18,9)) -> 2:float
+                        selectExpressions: CastDecimalToFloat(col 0:decimal(18,9)) -> 2:float
                     Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
                     Reduce Output Operator
                       key expressions: _col0 (type: float)
@@ -1650,11 +1650,11 @@ STAGE PLANS:
 PREHOOK: query: select cast(t as float) from decimal_2 order by t
 PREHOOK: type: QUERY
 PREHOOK: Input: default@decimal_2
-#### A masked pattern was here ####
+PREHOOK: Output: hdfs://### HDFS PATH ###
 POSTHOOK: query: select cast(t as float) from decimal_2 order by t
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@decimal_2
-#### A masked pattern was here ####
+POSTHOOK: Output: hdfs://### HDFS PATH ###
 3404045.5
 PREHOOK: query: explain vectorization detail
 select cast(t as double) from decimal_2 order by t
@@ -1766,11 +1766,11 @@ STAGE PLANS:
 PREHOOK: query: select cast(t as double) from decimal_2 order by t
 PREHOOK: type: QUERY
 PREHOOK: Input: default@decimal_2
-#### A masked pattern was here ####
+PREHOOK: Output: hdfs://### HDFS PATH ###
 POSTHOOK: query: select cast(t as double) from decimal_2 order by t
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@decimal_2
-#### A masked pattern was here ####
+POSTHOOK: Output: hdfs://### HDFS PATH ###
 3404045.5044003
 PREHOOK: query: explain vectorization detail
 select cast(t as string) from decimal_2 order by t
@@ -1882,11 +1882,11 @@ STAGE PLANS:
 PREHOOK: query: select cast(t as string) from decimal_2 order by t
 PREHOOK: type: QUERY
 PREHOOK: Input: default@decimal_2
-#### A masked pattern was here ####
+PREHOOK: Output: hdfs://### HDFS PATH ###
 POSTHOOK: query: select cast(t as string) from decimal_2 order by t
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@decimal_2
-#### A masked pattern was here ####
+POSTHOOK: Output: hdfs://### HDFS PATH ###
 3404045.5044003
 PREHOOK: query: explain vectorization detail
 select cast(3.14 as decimal(4,2)) as c from decimal_2 order by c
@@ -1961,11 +1961,11 @@ STAGE PLANS:
 PREHOOK: query: select cast(3.14 as decimal(4,2)) as c from decimal_2 order by c
 PREHOOK: type: QUERY
 PREHOOK: Input: default@decimal_2
-#### A masked pattern was here ####
+PREHOOK: Output: hdfs://### HDFS PATH ###
 POSTHOOK: query: select cast(3.14 as decimal(4,2)) as c from decimal_2 order by c
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@decimal_2
-#### A masked pattern was here ####
+POSTHOOK: Output: hdfs://### HDFS PATH ###
 3.14
 PREHOOK: query: explain vectorization detail
 select cast(cast(3.14 as float) as decimal(4,2)) as c from decimal_2 order by c
@@ -2040,11 +2040,11 @@ STAGE PLANS:
 PREHOOK: query: select cast(cast(3.14 as float) as decimal(4,2)) as c from decimal_2 order by c
 PREHOOK: type: QUERY
 PREHOOK: Input: default@decimal_2
-#### A masked pattern was here ####
+PREHOOK: Output: hdfs://### HDFS PATH ###
 POSTHOOK: query: select cast(cast(3.14 as float) as decimal(4,2)) as c from decimal_2 order by c
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@decimal_2
-#### A masked pattern was here ####
+POSTHOOK: Output: hdfs://### HDFS PATH ###
 3.14
 PREHOOK: query: explain vectorization detail
 select cast(cast('2012-12-19 11:12:19.1234567' as timestamp) as decimal(30,8)) as c from decimal_2 order by c
@@ -2119,11 +2119,11 @@ STAGE PLANS:
 PREHOOK: query: select cast(cast('2012-12-19 11:12:19.1234567' as timestamp) as decimal(30,8)) as c from decimal_2 order by c
 PREHOOK: type: QUERY
 PREHOOK: Input: default@decimal_2
-#### A masked pattern was here ####
+PREHOOK: Output: hdfs://### HDFS PATH ###
 POSTHOOK: query: select cast(cast('2012-12-19 11:12:19.1234567' as timestamp) as decimal(30,8)) as c from decimal_2 order by c
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@decimal_2
-#### A masked pattern was here ####
+POSTHOOK: Output: hdfs://### HDFS PATH ###
 1355944339.12345670
 PREHOOK: query: explain vectorization detail
 select cast(true as decimal) as c from decimal_2 order by c
@@ -2268,11 +2268,11 @@ STAGE PLANS:
 PREHOOK: query: select cast(true as decimal) as c from decimal_2 order by c
 PREHOOK: type: QUERY
 PREHOOK: Input: default@decimal_2
-#### A masked pattern was here ####
+PREHOOK: Output: hdfs://### HDFS PATH ###
 POSTHOOK: query: select cast(true as decimal) as c from decimal_2 order by c
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@decimal_2
-#### A masked pattern was here ####
+POSTHOOK: Output: hdfs://### HDFS PATH ###
 1
 PREHOOK: query: explain vectorization detail
 select cast(3Y as decimal) as c from decimal_2 order by c
@@ -2347,11 +2347,11 @@ STAGE PLANS:
 PREHOOK: query: select cast(3Y as decimal) as c from decimal_2 order by c
 PREHOOK: type: QUERY
 PREHOOK: Input: default@decimal_2
-#### A masked pattern was here ####
+PREHOOK: Output: hdfs://### HDFS PATH ###
 POSTHOOK: query: select cast(3Y as decimal) as c from decimal_2 order by c
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@decimal_2
-#### A masked pattern was here ####
+POSTHOOK: Output: hdfs://### HDFS PATH ###
 3
 PREHOOK: query: explain vectorization detail
 select cast(3S as decimal) as c from decimal_2 order by c
@@ -2426,11 +2426,11 @@ STAGE PLANS:
 PREHOOK: query: select cast(3S as decimal) as c from decimal_2 order by c
 PREHOOK: type: QUERY
 PREHOOK: Input: default@decimal_2
-#### A masked pattern was here ####
+PREHOOK: Output: hdfs://### HDFS PATH ###
 POSTHOOK: query: select cast(3S as decimal) as c from decimal_2 order by c
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@decimal_2
-#### A masked pattern was here ####
+POSTHOOK: Output: hdfs://### HDFS PATH ###
 3
 PREHOOK: query: explain vectorization detail
 select cast(cast(3 as int) as decimal) as c from decimal_2 order by c
@@ -2505,11 +2505,11 @@ STAGE PLANS:
 PREHOOK: query: select cast(cast(3 as int) as decimal) as c from decimal_2 order by c
 PREHOOK: type: QUERY
 PREHOOK: Input: default@decimal_2
-#### A masked pattern was here ####
+PREHOOK: Output: hdfs://### HDFS PATH ###
 POSTHOOK: query: select cast(cast(3 as int) as decimal) as c from decimal_2 order by c
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@decimal_2
-#### A masked pattern was here ####
+POSTHOOK: Output: hdfs://### HDFS PATH ###
 3
 PREHOOK: query: explain vectorization detail
 select cast(3L as decimal) as c from decimal_2 order by c
@@ -2584,11 +2584,11 @@ STAGE PLANS:
 PREHOOK: query: select cast(3L as decimal) as c from decimal_2 order by c
 PREHOOK: type: QUERY
 PREHOOK: Input: default@decimal_2
-#### A masked pattern was here ####
+PREHOOK: Output: hdfs://### HDFS PATH ###
 POSTHOOK: query: select cast(3L as decimal) as c from decimal_2 order by c
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@decimal_2
-#### A masked pattern was here ####
+POSTHOOK: Output: hdfs://### HDFS PATH ###
 3
 PREHOOK: query: explain vectorization detail
 select cast(0.99999999999999999999 as decimal(20,19)) as c from decimal_2 order by c
@@ -2663,11 +2663,11 @@ STAGE PLANS:
 PREHOOK: query: select cast(0.99999999999999999999 as decimal(20,19)) as c from decimal_2 order by c
 PREHOOK: type: QUERY
 PREHOOK: Input: default@decimal_2
-#### A masked pattern was here ####
+PREHOOK: Output: hdfs://### HDFS PATH ###
 POSTHOOK: query: select cast(0.99999999999999999999 as decimal(20,19)) as c from decimal_2 order by c
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@decimal_2
-#### A masked pattern was here ####
+POSTHOOK: Output: hdfs://### HDFS PATH ###
 1.0000000000000000000
 PREHOOK: query: explain vectorization detail
 select cast('0.99999999999999999999' as decimal(20,20)) as c from decimal_2 order by c
@@ -2742,11 +2742,11 @@ STAGE PLANS:
 PREHOOK: query: select cast('0.99999999999999999999' as decimal(20,20)) as c from decimal_2 order by c
 PREHOOK: type: QUERY
 PREHOOK: Input: default@decimal_2
-#### A masked pattern was here ####
+PREHOOK: Output: hdfs://### HDFS PATH ###
 POSTHOOK: query: select cast('0.99999999999999999999' as decimal(20,20)) as c from decimal_2 order by c
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@decimal_2
-#### A masked pattern was here ####
+POSTHOOK: Output: hdfs://### HDFS PATH ###
 0.99999999999999999999
 PREHOOK: query: drop table decimal_2
 PREHOOK: type: DROPTABLE

http://git-wip-us.apache.org/repos/asf/hive/blob/6936c9c2/ql/src/test/results/clientpositive/llap/vector_decimal_expressions.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/vector_decimal_expressions.q.out b/ql/src/test/results/clientpositive/llap/vector_decimal_expressions.q.out
index 64433ea..2ddc05e 100644
--- a/ql/src/test/results/clientpositive/llap/vector_decimal_expressions.q.out
+++ b/ql/src/test/results/clientpositive/llap/vector_decimal_expressions.q.out
@@ -76,7 +76,7 @@ STAGE PLANS:
                           className: VectorSelectOperator
                           native: true
                           projectedOutputColumnNums: [4, 6, 8, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20]
-                          selectExpressions: DecimalColAddDecimalColumn(col 1:decimal(20,10), col 2:decimal(23,14)) -> 4:decimal(25,14), DecimalColSubtractDecimalColumn(col 1:decimal(20,10), col 5:decimal(25,14))(children: DecimalScalarMultiplyDecimalColumn(val 2, col 2:decimal(23,14)) -> 5:decimal(25,14)) -> 6:decimal(26,14), DecimalColDivideDecimalColumn(col 7:decimal(21,10), col 2:decimal(23,14))(children: DecimalColAddDecimalScalar(col 1:decimal(20,10), val 2.34) -> 7:decimal(21,10)) -> 8:decimal(38,13), DecimalColMultiplyDecimalColumn(col 1:decimal(20,10), col 9:decimal(27,17))(children: DecimalColDivideDecimalScalar(col 2:decimal(23,14), val 3.4) -> 9:decimal(27,17)) -> 10:decimal(38,17), DecimalColModuloDecimalScalar(col 1:decimal(20,10), val 10) -> 11:decimal(12,10), CastDecimalToLong(col 1:decimal(20,10)) -> 12:int, CastDecimalToLong(col 2:decimal(23,14)) -> 13:smallint, CastDecimalToLong(col 2:decimal(23,14)) -> 14:tinyint, CastDecimalToLong(col 1:decimal(20,10)) -> 15:big
 int, CastDecimalToBoolean(col 1:decimal(20,10)) -> 16:boolean, CastDecimalToDouble(col 2:decimal(23,14)) -> 17:double, CastDecimalToDouble(col 1:decimal(20,10)) -> 18:float, CastDecimalToString(col 2:decimal(23,14)) -> 19:string, CastDecimalToTimestamp(col 1:decimal(20,10)) -> 20:timestamp
+                          selectExpressions: DecimalColAddDecimalColumn(col 1:decimal(20,10), col 2:decimal(23,14)) -> 4:decimal(25,14), DecimalColSubtractDecimalColumn(col 1:decimal(20,10), col 5:decimal(25,14))(children: DecimalScalarMultiplyDecimalColumn(val 2, col 2:decimal(23,14)) -> 5:decimal(25,14)) -> 6:decimal(26,14), DecimalColDivideDecimalColumn(col 7:decimal(21,10), col 2:decimal(23,14))(children: DecimalColAddDecimalScalar(col 1:decimal(20,10), val 2.34) -> 7:decimal(21,10)) -> 8:decimal(38,13), DecimalColMultiplyDecimalColumn(col 1:decimal(20,10), col 9:decimal(27,17))(children: DecimalColDivideDecimalScalar(col 2:decimal(23,14), val 3.4) -> 9:decimal(27,17)) -> 10:decimal(38,17), DecimalColModuloDecimalScalar(col 1:decimal(20,10), val 10) -> 11:decimal(12,10), CastDecimalToLong(col 1:decimal(20,10)) -> 12:int, CastDecimalToLong(col 2:decimal(23,14)) -> 13:smallint, CastDecimalToLong(col 2:decimal(23,14)) -> 14:tinyint, CastDecimalToLong(col 1:decimal(20,10)) -> 15:big
 int, CastDecimalToBoolean(col 1:decimal(20,10)) -> 16:boolean, CastDecimalToDouble(col 2:decimal(23,14)) -> 17:double, CastDecimalToFloat(col 1:decimal(20,10)) -> 18:float, CastDecimalToString(col 2:decimal(23,14)) -> 19:string, CastDecimalToTimestamp(col 1:decimal(20,10)) -> 20:timestamp
                       Statistics: Num rows: 455 Data size: 100294 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: decimal(25,14)), _col1 (type: decimal(26,14)), _col2 (type: decimal(38,13)), _col3 (type: decimal(38,17)), _col4 (type: decimal(12,10)), _col5 (type: int), _col6 (type: smallint), _col7 (type: tinyint), _col8 (type: bigint), _col9 (type: boolean), _col10 (type: double), _col11 (type: float), _col12 (type: string), _col13 (type: timestamp)
@@ -249,7 +249,7 @@ STAGE PLANS:
                           className: VectorSelectOperator
                           native: true
                           projectedOutputColumnNums: [4, 6, 8, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20]
-                          selectExpressions: DecimalColAddDecimalColumn(col 1:decimal(10,3), col 2:decimal(7,2)) -> 4:decimal(11,3), DecimalColSubtractDecimalColumn(col 1:decimal(10,3), col 5:decimal(9,2))(children: DecimalScalarMultiplyDecimalColumn(val 2, col 2:decimal(7,2)) -> 5:decimal(9,2)) -> 6:decimal(11,3), DecimalColDivideDecimalColumn(col 7:decimal(11,3), col 2:decimal(7,2))(children: DecimalColAddDecimalScalar(col 1:decimal(10,3), val 2.34) -> 7:decimal(11,3)) -> 8:decimal(21,11), DecimalColMultiplyDecimalColumn(col 1:decimal(10,3), col 9:decimal(12,6))(children: DecimalColDivideDecimalScalar(col 2:decimal(7,2), val 3.4) -> 9:decimal(12,6)) -> 10:decimal(23,9), DecimalColModuloDecimalScalar(col 1:decimal(10,3), val 10) -> 11:decimal(5,3), CastDecimalToLong(col 1:decimal(10,3)) -> 12:int, CastDecimalToLong(col 2:decimal(7,2)) -> 13:smallint, CastDecimalToLong(col 2:decimal(7,2)) -> 14:tinyint, CastDecimalToLong(col 1:decimal(10,3)) -> 15:bigint, CastDecimalToBoolean(col 1:
 decimal(10,3)) -> 16:boolean, CastDecimalToDouble(col 2:decimal(7,2)) -> 17:double, CastDecimalToDouble(col 1:decimal(10,3)) -> 18:float, CastDecimalToString(col 2:decimal(7,2)) -> 19:string, CastDecimalToTimestamp(col 1:decimal(10,3)) -> 20:timestamp
+                          selectExpressions: DecimalColAddDecimalColumn(col 1:decimal(10,3), col 2:decimal(7,2)) -> 4:decimal(11,3), DecimalColSubtractDecimalColumn(col 1:decimal(10,3), col 5:decimal(9,2))(children: DecimalScalarMultiplyDecimalColumn(val 2, col 2:decimal(7,2)) -> 5:decimal(9,2)) -> 6:decimal(11,3), DecimalColDivideDecimalColumn(col 7:decimal(11,3), col 2:decimal(7,2))(children: DecimalColAddDecimalScalar(col 1:decimal(10,3), val 2.34) -> 7:decimal(11,3)) -> 8:decimal(21,11), DecimalColMultiplyDecimalColumn(col 1:decimal(10,3), col 9:decimal(12,6))(children: DecimalColDivideDecimalScalar(col 2:decimal(7,2), val 3.4) -> 9:decimal(12,6)) -> 10:decimal(23,9), DecimalColModuloDecimalScalar(col 1:decimal(10,3), val 10) -> 11:decimal(5,3), CastDecimalToLong(col 1:decimal(10,3)) -> 12:int, CastDecimalToLong(col 2:decimal(7,2)) -> 13:smallint, CastDecimalToLong(col 2:decimal(7,2)) -> 14:tinyint, CastDecimalToLong(col 1:decimal(10,3)) -> 15:bigint, CastDecimalToBoolean(col 1:
 decimal(10,3)) -> 16:boolean, CastDecimalToDouble(col 2:decimal(7,2)) -> 17:double, CastDecimalToFloat(col 1:decimal(10,3)) -> 18:float, CastDecimalToString(col 2:decimal(7,2)) -> 19:string, CastDecimalToTimestamp(col 1:decimal(10,3)) -> 20:timestamp
                       Statistics: Num rows: 455 Data size: 100294 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: decimal(11,3)), _col1 (type: decimal(11,3)), _col2 (type: decimal(21,11)), _col3 (type: decimal(23,9)), _col4 (type: decimal(5,3)), _col5 (type: int), _col6 (type: smallint), _col7 (type: tinyint), _col8 (type: bigint), _col9 (type: boolean), _col10 (type: double), _col11 (type: float), _col12 (type: string), _col13 (type: timestamp)

http://git-wip-us.apache.org/repos/asf/hive/blob/6936c9c2/ql/src/test/results/clientpositive/vector_decimal_1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/vector_decimal_1.q.out b/ql/src/test/results/clientpositive/vector_decimal_1.q.out
index d4d4705..80def64 100644
--- a/ql/src/test/results/clientpositive/vector_decimal_1.q.out
+++ b/ql/src/test/results/clientpositive/vector_decimal_1.q.out
@@ -529,7 +529,7 @@ STAGE PLANS:
                   className: VectorSelectOperator
                   native: true
                   projectedOutputColumnNums: [4]
-                  selectExpressions: CastDecimalToDouble(col 0:decimal(4,2)) -> 4:float
+                  selectExpressions: CastDecimalToFloat(col 0:decimal(4,2)) -> 4:float
               Statistics: Num rows: 2 Data size: 336 Basic stats: COMPLETE Column stats: NONE
               Reduce Output Operator
                 key expressions: _col0 (type: float)

http://git-wip-us.apache.org/repos/asf/hive/blob/6936c9c2/ql/src/test/results/clientpositive/vector_decimal_expressions.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/vector_decimal_expressions.q.out b/ql/src/test/results/clientpositive/vector_decimal_expressions.q.out
index 2e4edc5..51ed896 100644
--- a/ql/src/test/results/clientpositive/vector_decimal_expressions.q.out
+++ b/ql/src/test/results/clientpositive/vector_decimal_expressions.q.out
@@ -70,7 +70,7 @@ STAGE PLANS:
                     className: VectorSelectOperator
                     native: true
                     projectedOutputColumnNums: [4, 6, 8, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20]
-                    selectExpressions: DecimalColAddDecimalColumn(col 1:decimal(20,10), col 2:decimal(23,14)) -> 4:decimal(25,14), DecimalColSubtractDecimalColumn(col 1:decimal(20,10), col 5:decimal(25,14))(children: DecimalScalarMultiplyDecimalColumn(val 2, col 2:decimal(23,14)) -> 5:decimal(25,14)) -> 6:decimal(26,14), DecimalColDivideDecimalColumn(col 7:decimal(21,10), col 2:decimal(23,14))(children: DecimalColAddDecimalScalar(col 1:decimal(20,10), val 2.34) -> 7:decimal(21,10)) -> 8:decimal(38,13), DecimalColMultiplyDecimalColumn(col 1:decimal(20,10), col 9:decimal(27,17))(children: DecimalColDivideDecimalScalar(col 2:decimal(23,14), val 3.4) -> 9:decimal(27,17)) -> 10:decimal(38,17), DecimalColModuloDecimalScalar(col 1:decimal(20,10), val 10) -> 11:decimal(12,10), CastDecimalToLong(col 1:decimal(20,10)) -> 12:int, CastDecimalToLong(col 2:decimal(23,14)) -> 13:smallint, CastDecimalToLong(col 2:decimal(23,14)) -> 14:tinyint, CastDecimalToLong(col 1:decimal(20,10)) -> 15:bigint, C
 astDecimalToBoolean(col 1:decimal(20,10)) -> 16:boolean, CastDecimalToDouble(col 2:decimal(23,14)) -> 17:double, CastDecimalToDouble(col 1:decimal(20,10)) -> 18:float, CastDecimalToString(col 2:decimal(23,14)) -> 19:string, CastDecimalToTimestamp(col 1:decimal(20,10)) -> 20:timestamp
+                    selectExpressions: DecimalColAddDecimalColumn(col 1:decimal(20,10), col 2:decimal(23,14)) -> 4:decimal(25,14), DecimalColSubtractDecimalColumn(col 1:decimal(20,10), col 5:decimal(25,14))(children: DecimalScalarMultiplyDecimalColumn(val 2, col 2:decimal(23,14)) -> 5:decimal(25,14)) -> 6:decimal(26,14), DecimalColDivideDecimalColumn(col 7:decimal(21,10), col 2:decimal(23,14))(children: DecimalColAddDecimalScalar(col 1:decimal(20,10), val 2.34) -> 7:decimal(21,10)) -> 8:decimal(38,13), DecimalColMultiplyDecimalColumn(col 1:decimal(20,10), col 9:decimal(27,17))(children: DecimalColDivideDecimalScalar(col 2:decimal(23,14), val 3.4) -> 9:decimal(27,17)) -> 10:decimal(38,17), DecimalColModuloDecimalScalar(col 1:decimal(20,10), val 10) -> 11:decimal(12,10), CastDecimalToLong(col 1:decimal(20,10)) -> 12:int, CastDecimalToLong(col 2:decimal(23,14)) -> 13:smallint, CastDecimalToLong(col 2:decimal(23,14)) -> 14:tinyint, CastDecimalToLong(col 1:decimal(20,10)) -> 15:bigint, C
 astDecimalToBoolean(col 1:decimal(20,10)) -> 16:boolean, CastDecimalToDouble(col 2:decimal(23,14)) -> 17:double, CastDecimalToFloat(col 1:decimal(20,10)) -> 18:float, CastDecimalToString(col 2:decimal(23,14)) -> 19:string, CastDecimalToTimestamp(col 1:decimal(20,10)) -> 20:timestamp
                 Statistics: Num rows: 455 Data size: 78802 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: decimal(25,14)), _col1 (type: decimal(26,14)), _col2 (type: decimal(38,13)), _col3 (type: decimal(38,17)), _col4 (type: decimal(12,10)), _col5 (type: int), _col6 (type: smallint), _col7 (type: tinyint), _col8 (type: bigint), _col9 (type: boolean), _col10 (type: double), _col11 (type: float), _col12 (type: string), _col13 (type: timestamp)
@@ -214,7 +214,7 @@ STAGE PLANS:
                     className: VectorSelectOperator
                     native: true
                     projectedOutputColumnNums: [4, 6, 8, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20]
-                    selectExpressions: DecimalColAddDecimalColumn(col 1:decimal(10,3), col 2:decimal(7,2)) -> 4:decimal(11,3), DecimalColSubtractDecimalColumn(col 1:decimal(10,3), col 5:decimal(9,2))(children: DecimalScalarMultiplyDecimalColumn(val 2, col 2:decimal(7,2)) -> 5:decimal(9,2)) -> 6:decimal(11,3), DecimalColDivideDecimalColumn(col 7:decimal(11,3), col 2:decimal(7,2))(children: DecimalColAddDecimalScalar(col 1:decimal(10,3), val 2.34) -> 7:decimal(11,3)) -> 8:decimal(21,11), DecimalColMultiplyDecimalColumn(col 1:decimal(10,3), col 9:decimal(12,6))(children: DecimalColDivideDecimalScalar(col 2:decimal(7,2), val 3.4) -> 9:decimal(12,6)) -> 10:decimal(23,9), DecimalColModuloDecimalScalar(col 1:decimal(10,3), val 10) -> 11:decimal(5,3), CastDecimalToLong(col 1:decimal(10,3)) -> 12:int, CastDecimalToLong(col 2:decimal(7,2)) -> 13:smallint, CastDecimalToLong(col 2:decimal(7,2)) -> 14:tinyint, CastDecimalToLong(col 1:decimal(10,3)) -> 15:bigint, CastDecimalToBoolean(col 1:decima
 l(10,3)) -> 16:boolean, CastDecimalToDouble(col 2:decimal(7,2)) -> 17:double, CastDecimalToDouble(col 1:decimal(10,3)) -> 18:float, CastDecimalToString(col 2:decimal(7,2)) -> 19:string, CastDecimalToTimestamp(col 1:decimal(10,3)) -> 20:timestamp
+                    selectExpressions: DecimalColAddDecimalColumn(col 1:decimal(10,3), col 2:decimal(7,2)) -> 4:decimal(11,3), DecimalColSubtractDecimalColumn(col 1:decimal(10,3), col 5:decimal(9,2))(children: DecimalScalarMultiplyDecimalColumn(val 2, col 2:decimal(7,2)) -> 5:decimal(9,2)) -> 6:decimal(11,3), DecimalColDivideDecimalColumn(col 7:decimal(11,3), col 2:decimal(7,2))(children: DecimalColAddDecimalScalar(col 1:decimal(10,3), val 2.34) -> 7:decimal(11,3)) -> 8:decimal(21,11), DecimalColMultiplyDecimalColumn(col 1:decimal(10,3), col 9:decimal(12,6))(children: DecimalColDivideDecimalScalar(col 2:decimal(7,2), val 3.4) -> 9:decimal(12,6)) -> 10:decimal(23,9), DecimalColModuloDecimalScalar(col 1:decimal(10,3), val 10) -> 11:decimal(5,3), CastDecimalToLong(col 1:decimal(10,3)) -> 12:int, CastDecimalToLong(col 2:decimal(7,2)) -> 13:smallint, CastDecimalToLong(col 2:decimal(7,2)) -> 14:tinyint, CastDecimalToLong(col 1:decimal(10,3)) -> 15:bigint, CastDecimalToBoolean(col 1:decima
 l(10,3)) -> 16:boolean, CastDecimalToDouble(col 2:decimal(7,2)) -> 17:double, CastDecimalToFloat(col 1:decimal(10,3)) -> 18:float, CastDecimalToString(col 2:decimal(7,2)) -> 19:string, CastDecimalToTimestamp(col 1:decimal(10,3)) -> 20:timestamp
                 Statistics: Num rows: 455 Data size: 78788 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: decimal(11,3)), _col1 (type: decimal(11,3)), _col2 (type: decimal(21,11)), _col3 (type: decimal(23,9)), _col4 (type: decimal(5,3)), _col5 (type: int), _col6 (type: smallint), _col7 (type: tinyint), _col8 (type: bigint), _col9 (type: boolean), _col10 (type: double), _col11 (type: float), _col12 (type: string), _col13 (type: timestamp)

http://git-wip-us.apache.org/repos/asf/hive/blob/6936c9c2/ql/src/test/results/clientpositive/vectorization_parquet_ppd_decimal.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/vectorization_parquet_ppd_decimal.q.out b/ql/src/test/results/clientpositive/vectorization_parquet_ppd_decimal.q.out
new file mode 100644
index 0000000..c2611fc
--- /dev/null
+++ b/ql/src/test/results/clientpositive/vectorization_parquet_ppd_decimal.q.out
@@ -0,0 +1,766 @@
+PREHOOK: query: create table newtypestbl(c char(10), v varchar(10), d decimal(5,3), da date) stored as parquet
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@newtypestbl
+POSTHOOK: query: create table newtypestbl(c char(10), v varchar(10), d decimal(5,3), da date) stored as parquet
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@newtypestbl
+PREHOOK: query: insert overwrite table newtypestbl select * from (select cast("apple" as char(10)), cast("bee" as varchar(10)), 0.22, cast("1970-02-20" as date) from src src1 union all select cast("hello" as char(10)), cast("world" as varchar(10)), 11.22, cast("1970-02-27" as date) from src src2 limit 10) uniontbl
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+PREHOOK: Output: default@newtypestbl
+POSTHOOK: query: insert overwrite table newtypestbl select * from (select cast("apple" as char(10)), cast("bee" as varchar(10)), 0.22, cast("1970-02-20" as date) from src src1 union all select cast("hello" as char(10)), cast("world" as varchar(10)), 11.22, cast("1970-02-27" as date) from src src2 limit 10) uniontbl
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+POSTHOOK: Output: default@newtypestbl
+POSTHOOK: Lineage: newtypestbl.c EXPRESSION []
+POSTHOOK: Lineage: newtypestbl.d EXPRESSION []
+POSTHOOK: Lineage: newtypestbl.da EXPRESSION []
+POSTHOOK: Lineage: newtypestbl.v EXPRESSION []
+PREHOOK: query: select * from newtypestbl where d=0.22
+PREHOOK: type: QUERY
+PREHOOK: Input: default@newtypestbl
+#### A masked pattern was here ####
+POSTHOOK: query: select * from newtypestbl where d=0.22
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@newtypestbl
+#### A masked pattern was here ####
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+PREHOOK: query: select * from newtypestbl where d=0.22
+PREHOOK: type: QUERY
+PREHOOK: Input: default@newtypestbl
+#### A masked pattern was here ####
+POSTHOOK: query: select * from newtypestbl where d=0.22
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@newtypestbl
+#### A masked pattern was here ####
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+PREHOOK: query: select * from newtypestbl where d='0.22'
+PREHOOK: type: QUERY
+PREHOOK: Input: default@newtypestbl
+#### A masked pattern was here ####
+POSTHOOK: query: select * from newtypestbl where d='0.22'
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@newtypestbl
+#### A masked pattern was here ####
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+PREHOOK: query: select * from newtypestbl where d='0.22'
+PREHOOK: type: QUERY
+PREHOOK: Input: default@newtypestbl
+#### A masked pattern was here ####
+POSTHOOK: query: select * from newtypestbl where d='0.22'
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@newtypestbl
+#### A masked pattern was here ####
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+PREHOOK: query: select * from newtypestbl where d=cast('0.22' as float)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@newtypestbl
+#### A masked pattern was here ####
+POSTHOOK: query: select * from newtypestbl where d=cast('0.22' as float)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@newtypestbl
+#### A masked pattern was here ####
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+PREHOOK: query: select * from newtypestbl where d=cast('0.22' as float)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@newtypestbl
+#### A masked pattern was here ####
+POSTHOOK: query: select * from newtypestbl where d=cast('0.22' as float)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@newtypestbl
+#### A masked pattern was here ####
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+PREHOOK: query: select * from newtypestbl where d!=0.22
+PREHOOK: type: QUERY
+PREHOOK: Input: default@newtypestbl
+#### A masked pattern was here ####
+POSTHOOK: query: select * from newtypestbl where d!=0.22
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@newtypestbl
+#### A masked pattern was here ####
+hello     	world	11.220	1970-02-27
+hello     	world	11.220	1970-02-27
+hello     	world	11.220	1970-02-27
+hello     	world	11.220	1970-02-27
+hello     	world	11.220	1970-02-27
+PREHOOK: query: select * from newtypestbl where d!=0.22
+PREHOOK: type: QUERY
+PREHOOK: Input: default@newtypestbl
+#### A masked pattern was here ####
+POSTHOOK: query: select * from newtypestbl where d!=0.22
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@newtypestbl
+#### A masked pattern was here ####
+hello     	world	11.220	1970-02-27
+hello     	world	11.220	1970-02-27
+hello     	world	11.220	1970-02-27
+hello     	world	11.220	1970-02-27
+hello     	world	11.220	1970-02-27
+PREHOOK: query: select * from newtypestbl where d!='0.22'
+PREHOOK: type: QUERY
+PREHOOK: Input: default@newtypestbl
+#### A masked pattern was here ####
+POSTHOOK: query: select * from newtypestbl where d!='0.22'
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@newtypestbl
+#### A masked pattern was here ####
+hello     	world	11.220	1970-02-27
+hello     	world	11.220	1970-02-27
+hello     	world	11.220	1970-02-27
+hello     	world	11.220	1970-02-27
+hello     	world	11.220	1970-02-27
+PREHOOK: query: select * from newtypestbl where d!='0.22'
+PREHOOK: type: QUERY
+PREHOOK: Input: default@newtypestbl
+#### A masked pattern was here ####
+POSTHOOK: query: select * from newtypestbl where d!='0.22'
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@newtypestbl
+#### A masked pattern was here ####
+hello     	world	11.220	1970-02-27
+hello     	world	11.220	1970-02-27
+hello     	world	11.220	1970-02-27
+hello     	world	11.220	1970-02-27
+hello     	world	11.220	1970-02-27
+PREHOOK: query: select * from newtypestbl where d!=cast('0.22' as float)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@newtypestbl
+#### A masked pattern was here ####
+POSTHOOK: query: select * from newtypestbl where d!=cast('0.22' as float)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@newtypestbl
+#### A masked pattern was here ####
+hello     	world	11.220	1970-02-27
+hello     	world	11.220	1970-02-27
+hello     	world	11.220	1970-02-27
+hello     	world	11.220	1970-02-27
+hello     	world	11.220	1970-02-27
+PREHOOK: query: select * from newtypestbl where d!=cast('0.22' as float)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@newtypestbl
+#### A masked pattern was here ####
+POSTHOOK: query: select * from newtypestbl where d!=cast('0.22' as float)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@newtypestbl
+#### A masked pattern was here ####
+hello     	world	11.220	1970-02-27
+hello     	world	11.220	1970-02-27
+hello     	world	11.220	1970-02-27
+hello     	world	11.220	1970-02-27
+hello     	world	11.220	1970-02-27
+PREHOOK: query: select * from newtypestbl where d<11.22
+PREHOOK: type: QUERY
+PREHOOK: Input: default@newtypestbl
+#### A masked pattern was here ####
+POSTHOOK: query: select * from newtypestbl where d<11.22
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@newtypestbl
+#### A masked pattern was here ####
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+PREHOOK: query: select * from newtypestbl where d<11.22
+PREHOOK: type: QUERY
+PREHOOK: Input: default@newtypestbl
+#### A masked pattern was here ####
+POSTHOOK: query: select * from newtypestbl where d<11.22
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@newtypestbl
+#### A masked pattern was here ####
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+PREHOOK: query: select * from newtypestbl where d<'11.22'
+PREHOOK: type: QUERY
+PREHOOK: Input: default@newtypestbl
+#### A masked pattern was here ####
+POSTHOOK: query: select * from newtypestbl where d<'11.22'
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@newtypestbl
+#### A masked pattern was here ####
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+PREHOOK: query: select * from newtypestbl where d<'11.22'
+PREHOOK: type: QUERY
+PREHOOK: Input: default@newtypestbl
+#### A masked pattern was here ####
+POSTHOOK: query: select * from newtypestbl where d<'11.22'
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@newtypestbl
+#### A masked pattern was here ####
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+PREHOOK: query: select * from newtypestbl where d<cast('11.22' as float)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@newtypestbl
+#### A masked pattern was here ####
+POSTHOOK: query: select * from newtypestbl where d<cast('11.22' as float)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@newtypestbl
+#### A masked pattern was here ####
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+PREHOOK: query: select * from newtypestbl where d<cast('11.22' as float)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@newtypestbl
+#### A masked pattern was here ####
+POSTHOOK: query: select * from newtypestbl where d<cast('11.22' as float)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@newtypestbl
+#### A masked pattern was here ####
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+PREHOOK: query: select * from newtypestbl where d<1
+PREHOOK: type: QUERY
+PREHOOK: Input: default@newtypestbl
+#### A masked pattern was here ####
+POSTHOOK: query: select * from newtypestbl where d<1
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@newtypestbl
+#### A masked pattern was here ####
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+PREHOOK: query: select * from newtypestbl where d<1
+PREHOOK: type: QUERY
+PREHOOK: Input: default@newtypestbl
+#### A masked pattern was here ####
+POSTHOOK: query: select * from newtypestbl where d<1
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@newtypestbl
+#### A masked pattern was here ####
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+PREHOOK: query: select * from newtypestbl where d<=11.22 sort by c
+PREHOOK: type: QUERY
+PREHOOK: Input: default@newtypestbl
+#### A masked pattern was here ####
+POSTHOOK: query: select * from newtypestbl where d<=11.22 sort by c
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@newtypestbl
+#### A masked pattern was here ####
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+hello     	world	11.220	1970-02-27
+hello     	world	11.220	1970-02-27
+hello     	world	11.220	1970-02-27
+hello     	world	11.220	1970-02-27
+hello     	world	11.220	1970-02-27
+PREHOOK: query: select * from newtypestbl where d<=11.22 sort by c
+PREHOOK: type: QUERY
+PREHOOK: Input: default@newtypestbl
+#### A masked pattern was here ####
+POSTHOOK: query: select * from newtypestbl where d<=11.22 sort by c
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@newtypestbl
+#### A masked pattern was here ####
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+hello     	world	11.220	1970-02-27
+hello     	world	11.220	1970-02-27
+hello     	world	11.220	1970-02-27
+hello     	world	11.220	1970-02-27
+hello     	world	11.220	1970-02-27
+PREHOOK: query: select * from newtypestbl where d<='11.22' sort by c
+PREHOOK: type: QUERY
+PREHOOK: Input: default@newtypestbl
+#### A masked pattern was here ####
+POSTHOOK: query: select * from newtypestbl where d<='11.22' sort by c
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@newtypestbl
+#### A masked pattern was here ####
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+hello     	world	11.220	1970-02-27
+hello     	world	11.220	1970-02-27
+hello     	world	11.220	1970-02-27
+hello     	world	11.220	1970-02-27
+hello     	world	11.220	1970-02-27
+PREHOOK: query: select * from newtypestbl where d<='11.22' sort by c
+PREHOOK: type: QUERY
+PREHOOK: Input: default@newtypestbl
+#### A masked pattern was here ####
+POSTHOOK: query: select * from newtypestbl where d<='11.22' sort by c
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@newtypestbl
+#### A masked pattern was here ####
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+hello     	world	11.220	1970-02-27
+hello     	world	11.220	1970-02-27
+hello     	world	11.220	1970-02-27
+hello     	world	11.220	1970-02-27
+hello     	world	11.220	1970-02-27
+PREHOOK: query: select * from newtypestbl where d<=cast('11.22' as float) sort by c
+PREHOOK: type: QUERY
+PREHOOK: Input: default@newtypestbl
+#### A masked pattern was here ####
+POSTHOOK: query: select * from newtypestbl where d<=cast('11.22' as float) sort by c
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@newtypestbl
+#### A masked pattern was here ####
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+hello     	world	11.220	1970-02-27
+hello     	world	11.220	1970-02-27
+hello     	world	11.220	1970-02-27
+hello     	world	11.220	1970-02-27
+hello     	world	11.220	1970-02-27
+PREHOOK: query: select * from newtypestbl where d<=cast('11.22' as float) sort by c
+PREHOOK: type: QUERY
+PREHOOK: Input: default@newtypestbl
+#### A masked pattern was here ####
+POSTHOOK: query: select * from newtypestbl where d<=cast('11.22' as float) sort by c
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@newtypestbl
+#### A masked pattern was here ####
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+hello     	world	11.220	1970-02-27
+hello     	world	11.220	1970-02-27
+hello     	world	11.220	1970-02-27
+hello     	world	11.220	1970-02-27
+hello     	world	11.220	1970-02-27
+PREHOOK: query: select * from newtypestbl where d<=cast('11.22' as decimal)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@newtypestbl
+#### A masked pattern was here ####
+POSTHOOK: query: select * from newtypestbl where d<=cast('11.22' as decimal)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@newtypestbl
+#### A masked pattern was here ####
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+PREHOOK: query: select * from newtypestbl where d<=cast('11.22' as decimal)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@newtypestbl
+#### A masked pattern was here ####
+POSTHOOK: query: select * from newtypestbl where d<=cast('11.22' as decimal)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@newtypestbl
+#### A masked pattern was here ####
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+PREHOOK: query: select * from newtypestbl where d<=11.22BD sort by c
+PREHOOK: type: QUERY
+PREHOOK: Input: default@newtypestbl
+#### A masked pattern was here ####
+POSTHOOK: query: select * from newtypestbl where d<=11.22BD sort by c
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@newtypestbl
+#### A masked pattern was here ####
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+hello     	world	11.220	1970-02-27
+hello     	world	11.220	1970-02-27
+hello     	world	11.220	1970-02-27
+hello     	world	11.220	1970-02-27
+hello     	world	11.220	1970-02-27
+PREHOOK: query: select * from newtypestbl where d<=11.22BD sort by c
+PREHOOK: type: QUERY
+PREHOOK: Input: default@newtypestbl
+#### A masked pattern was here ####
+POSTHOOK: query: select * from newtypestbl where d<=11.22BD sort by c
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@newtypestbl
+#### A masked pattern was here ####
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+hello     	world	11.220	1970-02-27
+hello     	world	11.220	1970-02-27
+hello     	world	11.220	1970-02-27
+hello     	world	11.220	1970-02-27
+hello     	world	11.220	1970-02-27
+PREHOOK: query: select * from newtypestbl where d<=12 sort by c
+PREHOOK: type: QUERY
+PREHOOK: Input: default@newtypestbl
+#### A masked pattern was here ####
+POSTHOOK: query: select * from newtypestbl where d<=12 sort by c
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@newtypestbl
+#### A masked pattern was here ####
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+hello     	world	11.220	1970-02-27
+hello     	world	11.220	1970-02-27
+hello     	world	11.220	1970-02-27
+hello     	world	11.220	1970-02-27
+hello     	world	11.220	1970-02-27
+PREHOOK: query: select * from newtypestbl where d<=12 sort by c
+PREHOOK: type: QUERY
+PREHOOK: Input: default@newtypestbl
+#### A masked pattern was here ####
+POSTHOOK: query: select * from newtypestbl where d<=12 sort by c
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@newtypestbl
+#### A masked pattern was here ####
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+hello     	world	11.220	1970-02-27
+hello     	world	11.220	1970-02-27
+hello     	world	11.220	1970-02-27
+hello     	world	11.220	1970-02-27
+hello     	world	11.220	1970-02-27
+PREHOOK: query: select * from newtypestbl where d in ('0.22', '1.0')
+PREHOOK: type: QUERY
+PREHOOK: Input: default@newtypestbl
+#### A masked pattern was here ####
+POSTHOOK: query: select * from newtypestbl where d in ('0.22', '1.0')
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@newtypestbl
+#### A masked pattern was here ####
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+PREHOOK: query: select * from newtypestbl where d in ('0.22', '1.0')
+PREHOOK: type: QUERY
+PREHOOK: Input: default@newtypestbl
+#### A masked pattern was here ####
+POSTHOOK: query: select * from newtypestbl where d in ('0.22', '1.0')
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@newtypestbl
+#### A masked pattern was here ####
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+PREHOOK: query: select * from newtypestbl where d in ('0.22', '11.22') sort by c
+PREHOOK: type: QUERY
+PREHOOK: Input: default@newtypestbl
+#### A masked pattern was here ####
+POSTHOOK: query: select * from newtypestbl where d in ('0.22', '11.22') sort by c
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@newtypestbl
+#### A masked pattern was here ####
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+hello     	world	11.220	1970-02-27
+hello     	world	11.220	1970-02-27
+hello     	world	11.220	1970-02-27
+hello     	world	11.220	1970-02-27
+hello     	world	11.220	1970-02-27
+PREHOOK: query: select * from newtypestbl where d in ('0.22', '11.22') sort by c
+PREHOOK: type: QUERY
+PREHOOK: Input: default@newtypestbl
+#### A masked pattern was here ####
+POSTHOOK: query: select * from newtypestbl where d in ('0.22', '11.22') sort by c
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@newtypestbl
+#### A masked pattern was here ####
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+hello     	world	11.220	1970-02-27
+hello     	world	11.220	1970-02-27
+hello     	world	11.220	1970-02-27
+hello     	world	11.220	1970-02-27
+hello     	world	11.220	1970-02-27
+PREHOOK: query: select * from newtypestbl where d in ('0.9', '1.0')
+PREHOOK: type: QUERY
+PREHOOK: Input: default@newtypestbl
+#### A masked pattern was here ####
+POSTHOOK: query: select * from newtypestbl where d in ('0.9', '1.0')
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@newtypestbl
+#### A masked pattern was here ####
+PREHOOK: query: select * from newtypestbl where d in ('0.9', '1.0')
+PREHOOK: type: QUERY
+PREHOOK: Input: default@newtypestbl
+#### A masked pattern was here ####
+POSTHOOK: query: select * from newtypestbl where d in ('0.9', '1.0')
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@newtypestbl
+#### A masked pattern was here ####
+PREHOOK: query: select * from newtypestbl where d in ('0.9', 0.22)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@newtypestbl
+#### A masked pattern was here ####
+POSTHOOK: query: select * from newtypestbl where d in ('0.9', 0.22)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@newtypestbl
+#### A masked pattern was here ####
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+PREHOOK: query: select * from newtypestbl where d in ('0.9', 0.22)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@newtypestbl
+#### A masked pattern was here ####
+POSTHOOK: query: select * from newtypestbl where d in ('0.9', 0.22)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@newtypestbl
+#### A masked pattern was here ####
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+PREHOOK: query: select * from newtypestbl where d in ('0.9', 0.22, cast('11.22' as float)) sort by c
+PREHOOK: type: QUERY
+PREHOOK: Input: default@newtypestbl
+#### A masked pattern was here ####
+POSTHOOK: query: select * from newtypestbl where d in ('0.9', 0.22, cast('11.22' as float)) sort by c
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@newtypestbl
+#### A masked pattern was here ####
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+hello     	world	11.220	1970-02-27
+hello     	world	11.220	1970-02-27
+hello     	world	11.220	1970-02-27
+hello     	world	11.220	1970-02-27
+hello     	world	11.220	1970-02-27
+PREHOOK: query: select * from newtypestbl where d in ('0.9', 0.22, cast('11.22' as float)) sort by c
+PREHOOK: type: QUERY
+PREHOOK: Input: default@newtypestbl
+#### A masked pattern was here ####
+POSTHOOK: query: select * from newtypestbl where d in ('0.9', 0.22, cast('11.22' as float)) sort by c
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@newtypestbl
+#### A masked pattern was here ####
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+hello     	world	11.220	1970-02-27
+hello     	world	11.220	1970-02-27
+hello     	world	11.220	1970-02-27
+hello     	world	11.220	1970-02-27
+hello     	world	11.220	1970-02-27
+PREHOOK: query: select * from newtypestbl where d between 0 and 1
+PREHOOK: type: QUERY
+PREHOOK: Input: default@newtypestbl
+#### A masked pattern was here ####
+POSTHOOK: query: select * from newtypestbl where d between 0 and 1
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@newtypestbl
+#### A masked pattern was here ####
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+PREHOOK: query: select * from newtypestbl where d between 0 and 1
+PREHOOK: type: QUERY
+PREHOOK: Input: default@newtypestbl
+#### A masked pattern was here ####
+POSTHOOK: query: select * from newtypestbl where d between 0 and 1
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@newtypestbl
+#### A masked pattern was here ####
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+PREHOOK: query: select * from newtypestbl where d between 0 and 1000 sort by c
+PREHOOK: type: QUERY
+PREHOOK: Input: default@newtypestbl
+#### A masked pattern was here ####
+POSTHOOK: query: select * from newtypestbl where d between 0 and 1000 sort by c
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@newtypestbl
+#### A masked pattern was here ####
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+hello     	world	11.220	1970-02-27
+hello     	world	11.220	1970-02-27
+hello     	world	11.220	1970-02-27
+hello     	world	11.220	1970-02-27
+hello     	world	11.220	1970-02-27
+PREHOOK: query: select * from newtypestbl where d between 0 and 1000 sort by c
+PREHOOK: type: QUERY
+PREHOOK: Input: default@newtypestbl
+#### A masked pattern was here ####
+POSTHOOK: query: select * from newtypestbl where d between 0 and 1000 sort by c
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@newtypestbl
+#### A masked pattern was here ####
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+hello     	world	11.220	1970-02-27
+hello     	world	11.220	1970-02-27
+hello     	world	11.220	1970-02-27
+hello     	world	11.220	1970-02-27
+hello     	world	11.220	1970-02-27
+PREHOOK: query: select * from newtypestbl where d between 0 and '2.0'
+PREHOOK: type: QUERY
+PREHOOK: Input: default@newtypestbl
+#### A masked pattern was here ####
+POSTHOOK: query: select * from newtypestbl where d between 0 and '2.0'
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@newtypestbl
+#### A masked pattern was here ####
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+PREHOOK: query: select * from newtypestbl where d between 0 and '2.0'
+PREHOOK: type: QUERY
+PREHOOK: Input: default@newtypestbl
+#### A masked pattern was here ####
+POSTHOOK: query: select * from newtypestbl where d between 0 and '2.0'
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@newtypestbl
+#### A masked pattern was here ####
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+PREHOOK: query: select * from newtypestbl where d between 0 and cast(3 as float)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@newtypestbl
+#### A masked pattern was here ####
+POSTHOOK: query: select * from newtypestbl where d between 0 and cast(3 as float)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@newtypestbl
+#### A masked pattern was here ####
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+PREHOOK: query: select * from newtypestbl where d between 0 and cast(3 as float)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@newtypestbl
+#### A masked pattern was here ####
+POSTHOOK: query: select * from newtypestbl where d between 0 and cast(3 as float)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@newtypestbl
+#### A masked pattern was here ####
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+apple     	bee	0.220	1970-02-20
+PREHOOK: query: select * from newtypestbl where d between 1 and cast(30 as char(10))
+PREHOOK: type: QUERY
+PREHOOK: Input: default@newtypestbl
+#### A masked pattern was here ####
+POSTHOOK: query: select * from newtypestbl where d between 1 and cast(30 as char(10))
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@newtypestbl
+#### A masked pattern was here ####
+hello     	world	11.220	1970-02-27
+hello     	world	11.220	1970-02-27
+hello     	world	11.220	1970-02-27
+hello     	world	11.220	1970-02-27
+hello     	world	11.220	1970-02-27
+PREHOOK: query: select * from newtypestbl where d between 1 and cast(30 as char(10))
+PREHOOK: type: QUERY
+PREHOOK: Input: default@newtypestbl
+#### A masked pattern was here ####
+POSTHOOK: query: select * from newtypestbl where d between 1 and cast(30 as char(10))
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@newtypestbl
+#### A masked pattern was here ####
+hello     	world	11.220	1970-02-27
+hello     	world	11.220	1970-02-27
+hello     	world	11.220	1970-02-27
+hello     	world	11.220	1970-02-27
+hello     	world	11.220	1970-02-27


[22/50] [abbrv] hive git commit: HIVE-18193: Migrate existing ACID tables to use write id per table rather than global transaction id (Sankar Hariappan, reviewed by Mahesh Kumar Behera, Thejas M Nair, Eugene Koifman)

Posted by vg...@apache.org.
HIVE-18193: Migrate existing ACID tables to use write id per table rather than global transaction id (Sankar Hariappan, reviewed by Mahesh Kumar Behera, Thejas M Nair, Eugene Koifman)


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

Branch: refs/heads/branch-3.0.0
Commit: 11a7164f0b4c54fc1e9f54f842719537abb6ac53
Parents: 32e29cc
Author: Sankar Hariappan <sa...@apache.org>
Authored: Sat May 12 00:29:16 2018 +0530
Committer: Sankar Hariappan <sa...@apache.org>
Committed: Sat May 12 00:29:16 2018 +0530

----------------------------------------------------------------------
 .../upgrade/derby/057-HIVE-18193.derby.sql      | 24 ++++++++++++
 .../hadoop/hive/metastore/txn/TxnDbUtil.java    |  4 +-
 .../main/sql/derby/hive-schema-3.0.0.derby.sql  |  4 +-
 .../sql/derby/upgrade-2.3.0-to-3.0.0.derby.sql  | 25 +++++++++++++
 .../main/sql/mssql/hive-schema-3.0.0.mssql.sql  |  4 +-
 .../sql/mssql/upgrade-2.3.0-to-3.0.0.mssql.sql  | 25 +++++++++++++
 .../main/sql/mysql/hive-schema-3.0.0.mysql.sql  |  2 +-
 .../sql/mysql/upgrade-2.3.0-to-3.0.0.mysql.sql  | 27 ++++++++++++++
 .../sql/oracle/hive-schema-3.0.0.oracle.sql     | 22 +++++------
 .../oracle/upgrade-2.3.0-to-3.0.0.oracle.sql    | 39 ++++++++++++++++----
 .../sql/postgres/hive-schema-3.0.0.postgres.sql |  4 +-
 .../upgrade-2.3.0-to-3.0.0.postgres.sql         | 25 +++++++++++++
 12 files changed, 178 insertions(+), 27 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/11a7164f/metastore/scripts/upgrade/derby/057-HIVE-18193.derby.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/derby/057-HIVE-18193.derby.sql b/metastore/scripts/upgrade/derby/057-HIVE-18193.derby.sql
new file mode 100644
index 0000000..499d06e
--- /dev/null
+++ b/metastore/scripts/upgrade/derby/057-HIVE-18193.derby.sql
@@ -0,0 +1,24 @@
+
+-- Populate NEXT_WRITE_ID for each Transactional table and set next write ID same as next txn ID
+INSERT INTO NEXT_WRITE_ID (NWI_DATABASE, NWI_TABLE, NWI_NEXT)
+    SELECT * FROM
+        (SELECT DB.NAME, TBL_INFO.TBL_NAME FROM DBS DB,
+            (SELECT TBL.DB_ID, TBL.TBL_NAME FROM TBLS TBL,
+                (SELECT TBL_ID FROM TABLE_PARAMS WHERE PARAM_KEY='transactional' AND CAST(PARAM_VALUE AS VARCHAR(128))='true') TBL_PARAM
+            WHERE TBL.TBL_ID=TBL_PARAM.TBL_ID) TBL_INFO
+        where DB.DB_ID=TBL_INFO.DB_ID) DB_TBL_NAME,
+        (SELECT NTXN_NEXT FROM NEXT_TXN_ID) NEXT_TXN_ID;
+
+-- Populate TXN_TO_WRITE_ID for each aborted/open txns and set write ID equal to txn ID
+INSERT INTO TXN_TO_WRITE_ID (T2W_DATABASE, T2W_TABLE, T2W_TXNID, T2W_WRITEID)
+    SELECT * FROM
+        (SELECT DB.NAME, TBL_INFO.TBL_NAME FROM DBS DB,
+            (SELECT TBL.DB_ID, TBL.TBL_NAME FROM TBLS TBL,
+                (SELECT TBL_ID FROM TABLE_PARAMS WHERE PARAM_KEY='transactional' AND CAST(PARAM_VALUE AS VARCHAR(128))='true') TBL_PARAM
+            WHERE TBL.TBL_ID=TBL_PARAM.TBL_ID) TBL_INFO
+        where DB.DB_ID=TBL_INFO.DB_ID) DB_TBL_NAME,
+        (SELECT TXN_ID, TXN_ID as WRITE_ID FROM TXNS) TXN_INFO;
+
+-- Update TXN_COMPONENTS and COMPLETED_TXN_COMPONENTS for write ID which is same as txn ID
+UPDATE TXN_COMPONENTS SET TC_WRITEID = TC_TXNID;
+UPDATE COMPLETED_TXN_COMPONENTS SET CTC_WRITEID = CTC_TXNID;

http://git-wip-us.apache.org/repos/asf/hive/blob/11a7164f/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnDbUtil.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnDbUtil.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnDbUtil.java
index cf89ab2..4597166 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnDbUtil.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnDbUtil.java
@@ -82,14 +82,14 @@ public final class TxnDbUtil {
           "  TXN_HOST varchar(128) NOT NULL)");
 
       stmt.execute("CREATE TABLE TXN_COMPONENTS (" +
-          "  TC_TXNID bigint REFERENCES TXNS (TXN_ID)," +
+          "  TC_TXNID bigint NOT NULL REFERENCES TXNS (TXN_ID)," +
           "  TC_DATABASE varchar(128) NOT NULL," +
           "  TC_TABLE varchar(128)," +
           "  TC_PARTITION varchar(767)," +
           "  TC_OPERATION_TYPE char(1) NOT NULL," +
           "  TC_WRITEID bigint)");
       stmt.execute("CREATE TABLE COMPLETED_TXN_COMPONENTS (" +
-          "  CTC_TXNID bigint," +
+          "  CTC_TXNID bigint NOT NULL," +
           "  CTC_DATABASE varchar(128) NOT NULL," +
           "  CTC_TABLE varchar(128)," +
           "  CTC_PARTITION varchar(767)," +

http://git-wip-us.apache.org/repos/asf/hive/blob/11a7164f/standalone-metastore/src/main/sql/derby/hive-schema-3.0.0.derby.sql
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/sql/derby/hive-schema-3.0.0.derby.sql b/standalone-metastore/src/main/sql/derby/hive-schema-3.0.0.derby.sql
index 8e09755..e818e1b 100644
--- a/standalone-metastore/src/main/sql/derby/hive-schema-3.0.0.derby.sql
+++ b/standalone-metastore/src/main/sql/derby/hive-schema-3.0.0.derby.sql
@@ -507,7 +507,7 @@ CREATE TABLE TXNS (
 );
 
 CREATE TABLE TXN_COMPONENTS (
-  TC_TXNID bigint REFERENCES TXNS (TXN_ID),
+  TC_TXNID bigint NOT NULL REFERENCES TXNS (TXN_ID),
   TC_DATABASE varchar(128) NOT NULL,
   TC_TABLE varchar(128),
   TC_PARTITION varchar(767),
@@ -518,7 +518,7 @@ CREATE TABLE TXN_COMPONENTS (
 CREATE INDEX TC_TXNID_INDEX ON TXN_COMPONENTS (TC_TXNID);
 
 CREATE TABLE COMPLETED_TXN_COMPONENTS (
-  CTC_TXNID bigint,
+  CTC_TXNID bigint NOT NULL,
   CTC_DATABASE varchar(128) NOT NULL,
   CTC_TABLE varchar(256),
   CTC_PARTITION varchar(767),

http://git-wip-us.apache.org/repos/asf/hive/blob/11a7164f/standalone-metastore/src/main/sql/derby/upgrade-2.3.0-to-3.0.0.derby.sql
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/sql/derby/upgrade-2.3.0-to-3.0.0.derby.sql b/standalone-metastore/src/main/sql/derby/upgrade-2.3.0-to-3.0.0.derby.sql
index 73bef36..7b7a8a2 100644
--- a/standalone-metastore/src/main/sql/derby/upgrade-2.3.0-to-3.0.0.derby.sql
+++ b/standalone-metastore/src/main/sql/derby/upgrade-2.3.0-to-3.0.0.derby.sql
@@ -254,6 +254,31 @@ CREATE TABLE "APP"."RUNTIME_STATS" (
 
 CREATE INDEX IDX_RUNTIME_STATS_CREATE_TIME ON RUNTIME_STATS(CREATE_TIME);
 
+-- HIVE-18193
+-- Populate NEXT_WRITE_ID for each Transactional table and set next write ID same as next txn ID
+INSERT INTO NEXT_WRITE_ID (NWI_DATABASE, NWI_TABLE, NWI_NEXT)
+    SELECT * FROM
+        (SELECT DB.NAME, TBL_INFO.TBL_NAME FROM DBS DB,
+            (SELECT TBL.DB_ID, TBL.TBL_NAME FROM TBLS TBL,
+                (SELECT TBL_ID FROM TABLE_PARAMS WHERE PARAM_KEY='transactional' AND CAST(PARAM_VALUE AS VARCHAR(128))='true') TBL_PARAM
+            WHERE TBL.TBL_ID=TBL_PARAM.TBL_ID) TBL_INFO
+        where DB.DB_ID=TBL_INFO.DB_ID) DB_TBL_NAME,
+        (SELECT NTXN_NEXT FROM NEXT_TXN_ID) NEXT_WRITE;
+
+-- Populate TXN_TO_WRITE_ID for each aborted/open txns and set write ID equal to txn ID
+INSERT INTO TXN_TO_WRITE_ID (T2W_DATABASE, T2W_TABLE, T2W_TXNID, T2W_WRITEID)
+    SELECT * FROM
+        (SELECT DB.NAME, TBL_INFO.TBL_NAME FROM DBS DB,
+            (SELECT TBL.DB_ID, TBL.TBL_NAME FROM TBLS TBL,
+                (SELECT TBL_ID FROM TABLE_PARAMS WHERE PARAM_KEY='transactional' AND CAST(PARAM_VALUE AS VARCHAR(128))='true') TBL_PARAM
+            WHERE TBL.TBL_ID=TBL_PARAM.TBL_ID) TBL_INFO
+        where DB.DB_ID=TBL_INFO.DB_ID) DB_TBL_NAME,
+        (SELECT TXN_ID, TXN_ID as WRITE_ID FROM TXNS) TXN_INFO;
+
+-- Update TXN_COMPONENTS and COMPLETED_TXN_COMPONENTS for write ID which is same as txn ID
+UPDATE TXN_COMPONENTS SET TC_WRITEID = TC_TXNID;
+UPDATE COMPLETED_TXN_COMPONENTS SET CTC_WRITEID = CTC_TXNID;
+
 -- This needs to be the last thing done.  Insert any changes above this line.
 UPDATE "APP".VERSION SET SCHEMA_VERSION='3.0.0', VERSION_COMMENT='Hive release version 3.0.0' where VER_ID=1;
 

http://git-wip-us.apache.org/repos/asf/hive/blob/11a7164f/standalone-metastore/src/main/sql/mssql/hive-schema-3.0.0.mssql.sql
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/sql/mssql/hive-schema-3.0.0.mssql.sql b/standalone-metastore/src/main/sql/mssql/hive-schema-3.0.0.mssql.sql
index 51df92c..c88fb18 100644
--- a/standalone-metastore/src/main/sql/mssql/hive-schema-3.0.0.mssql.sql
+++ b/standalone-metastore/src/main/sql/mssql/hive-schema-3.0.0.mssql.sql
@@ -1022,7 +1022,7 @@ PRIMARY KEY CLUSTERED
 );
 
 CREATE TABLE COMPLETED_TXN_COMPONENTS(
-	CTC_TXNID bigint NULL,
+	CTC_TXNID bigint NOT NULL,
 	CTC_DATABASE nvarchar(128) NOT NULL,
 	CTC_TABLE nvarchar(128) NULL,
 	CTC_PARTITION nvarchar(767) NULL,
@@ -1091,7 +1091,7 @@ PRIMARY KEY CLUSTERED
 );
 
 CREATE TABLE TXN_COMPONENTS(
-	TC_TXNID bigint NULL,
+	TC_TXNID bigint NOT NULL,
 	TC_DATABASE nvarchar(128) NOT NULL,
 	TC_TABLE nvarchar(128) NULL,
 	TC_PARTITION nvarchar(767) NULL,

http://git-wip-us.apache.org/repos/asf/hive/blob/11a7164f/standalone-metastore/src/main/sql/mssql/upgrade-2.3.0-to-3.0.0.mssql.sql
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/sql/mssql/upgrade-2.3.0-to-3.0.0.mssql.sql b/standalone-metastore/src/main/sql/mssql/upgrade-2.3.0-to-3.0.0.mssql.sql
index a7232dd..14c3deb 100644
--- a/standalone-metastore/src/main/sql/mssql/upgrade-2.3.0-to-3.0.0.mssql.sql
+++ b/standalone-metastore/src/main/sql/mssql/upgrade-2.3.0-to-3.0.0.mssql.sql
@@ -321,6 +321,31 @@ CREATE TABLE RUNTIME_STATS (
 
 CREATE INDEX IDX_RUNTIME_STATS_CREATE_TIME ON RUNTIME_STATS(CREATE_TIME);
 
+-- HIVE-18193
+-- Populate NEXT_WRITE_ID for each Transactional table and set next write ID same as next txn ID
+INSERT INTO NEXT_WRITE_ID (NWI_DATABASE, NWI_TABLE, NWI_NEXT)
+    SELECT * FROM
+        (SELECT DB.NAME, TBL_INFO.TBL_NAME FROM DBS DB,
+            (SELECT TBL.DB_ID, TBL.TBL_NAME FROM TBLS TBL,
+                (SELECT TBL_ID FROM TABLE_PARAMS WHERE PARAM_KEY='transactional' AND PARAM_VALUE='true') TBL_PARAM
+            WHERE TBL.TBL_ID=TBL_PARAM.TBL_ID) TBL_INFO
+        where DB.DB_ID=TBL_INFO.DB_ID) DB_TBL_NAME,
+        (SELECT NTXN_NEXT FROM NEXT_TXN_ID) NEXT_WRITE;
+
+-- Populate TXN_TO_WRITE_ID for each aborted/open txns and set write ID equal to txn ID
+INSERT INTO TXN_TO_WRITE_ID (T2W_DATABASE, T2W_TABLE, T2W_TXNID, T2W_WRITEID)
+    SELECT * FROM
+        (SELECT DB.NAME, TBL_INFO.TBL_NAME FROM DBS DB,
+            (SELECT TBL.DB_ID, TBL.TBL_NAME FROM TBLS TBL,
+                (SELECT TBL_ID FROM TABLE_PARAMS WHERE PARAM_KEY='transactional' AND PARAM_VALUE='true') TBL_PARAM
+            WHERE TBL.TBL_ID=TBL_PARAM.TBL_ID) TBL_INFO
+        where DB.DB_ID=TBL_INFO.DB_ID) DB_TBL_NAME,
+        (SELECT TXN_ID, TXN_ID as WRITE_ID FROM TXNS) TXN_INFO;
+
+-- Update TXN_COMPONENTS and COMPLETED_TXN_COMPONENTS for write ID which is same as txn ID
+UPDATE TXN_COMPONENTS SET TC_WRITEID = TC_TXNID;
+UPDATE COMPLETED_TXN_COMPONENTS SET CTC_WRITEID = CTC_TXNID;
+
 -- These lines need to be last.  Insert any changes above.
 UPDATE VERSION SET SCHEMA_VERSION='3.0.0', VERSION_COMMENT='Hive release version 3.0.0' where VER_ID=1;
 SELECT 'Finished upgrading MetaStore schema from 2.3.0 to 3.0.0' AS MESSAGE;

http://git-wip-us.apache.org/repos/asf/hive/blob/11a7164f/standalone-metastore/src/main/sql/mysql/hive-schema-3.0.0.mysql.sql
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/sql/mysql/hive-schema-3.0.0.mysql.sql b/standalone-metastore/src/main/sql/mysql/hive-schema-3.0.0.mysql.sql
index 8e55e94..c54df55 100644
--- a/standalone-metastore/src/main/sql/mysql/hive-schema-3.0.0.mysql.sql
+++ b/standalone-metastore/src/main/sql/mysql/hive-schema-3.0.0.mysql.sql
@@ -983,7 +983,7 @@ CREATE TABLE TXNS (
 CREATE TABLE TXN_COMPONENTS (
   TC_TXNID bigint NOT NULL,
   TC_DATABASE varchar(128) NOT NULL,
-  TC_TABLE varchar(128) NOT NULL,
+  TC_TABLE varchar(128),
   TC_PARTITION varchar(767),
   TC_OPERATION_TYPE char(1) NOT NULL,
   TC_WRITEID bigint,

http://git-wip-us.apache.org/repos/asf/hive/blob/11a7164f/standalone-metastore/src/main/sql/mysql/upgrade-2.3.0-to-3.0.0.mysql.sql
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/sql/mysql/upgrade-2.3.0-to-3.0.0.mysql.sql b/standalone-metastore/src/main/sql/mysql/upgrade-2.3.0-to-3.0.0.mysql.sql
index 9a48346..9b87563 100644
--- a/standalone-metastore/src/main/sql/mysql/upgrade-2.3.0-to-3.0.0.mysql.sql
+++ b/standalone-metastore/src/main/sql/mysql/upgrade-2.3.0-to-3.0.0.mysql.sql
@@ -292,6 +292,33 @@ CREATE TABLE RUNTIME_STATS (
 
 CREATE INDEX IDX_RUNTIME_STATS_CREATE_TIME ON RUNTIME_STATS(CREATE_TIME);
 
+-- HIVE-18193
+-- Populate NEXT_WRITE_ID for each Transactional table and set next write ID same as next txn ID
+INSERT INTO NEXT_WRITE_ID (NWI_DATABASE, NWI_TABLE, NWI_NEXT)
+    SELECT * FROM
+        (SELECT DB.NAME, TBL_INFO.TBL_NAME FROM DBS DB,
+            (SELECT TBL.DB_ID, TBL.TBL_NAME FROM TBLS TBL,
+                (SELECT TBL_ID FROM TABLE_PARAMS WHERE PARAM_KEY='transactional' AND PARAM_VALUE='true') TBL_PARAM
+            WHERE TBL.TBL_ID=TBL_PARAM.TBL_ID) TBL_INFO
+        where DB.DB_ID=TBL_INFO.DB_ID) DB_TBL_NAME,
+        (SELECT NTXN_NEXT FROM NEXT_TXN_ID) NEXT_WRITE;
+
+-- Populate TXN_TO_WRITE_ID for each aborted/open txns and set write ID equal to txn ID
+INSERT INTO TXN_TO_WRITE_ID (T2W_DATABASE, T2W_TABLE, T2W_TXNID, T2W_WRITEID)
+    SELECT * FROM
+        (SELECT DB.NAME, TBL_INFO.TBL_NAME FROM DBS DB,
+            (SELECT TBL.DB_ID, TBL.TBL_NAME FROM TBLS TBL,
+                (SELECT TBL_ID FROM TABLE_PARAMS WHERE PARAM_KEY='transactional' AND PARAM_VALUE='true') TBL_PARAM
+            WHERE TBL.TBL_ID=TBL_PARAM.TBL_ID) TBL_INFO
+        where DB.DB_ID=TBL_INFO.DB_ID) DB_TBL_NAME,
+        (SELECT TXN_ID, TXN_ID as WRITE_ID FROM TXNS) TXN_INFO;
+
+-- Update TXN_COMPONENTS and COMPLETED_TXN_COMPONENTS for write ID which is same as txn ID
+UPDATE TXN_COMPONENTS SET TC_WRITEID = TC_TXNID;
+UPDATE COMPLETED_TXN_COMPONENTS SET CTC_WRITEID = CTC_TXNID;
+
+ALTER TABLE TXN_COMPONENTS MODIFY COLUMN TC_TABLE varchar(128) NULL;
+
 -- These lines need to be last.  Insert any changes above.
 UPDATE VERSION SET SCHEMA_VERSION='3.0.0', VERSION_COMMENT='Hive release version 3.0.0' where VER_ID=1;
 SELECT 'Finished upgrading MetaStore schema from 2.3.0 to 3.0.0' AS ' ';

http://git-wip-us.apache.org/repos/asf/hive/blob/11a7164f/standalone-metastore/src/main/sql/oracle/hive-schema-3.0.0.oracle.sql
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/sql/oracle/hive-schema-3.0.0.oracle.sql b/standalone-metastore/src/main/sql/oracle/hive-schema-3.0.0.oracle.sql
index 3a12e08..63cc1f7 100644
--- a/standalone-metastore/src/main/sql/oracle/hive-schema-3.0.0.oracle.sql
+++ b/standalone-metastore/src/main/sql/oracle/hive-schema-3.0.0.oracle.sql
@@ -958,7 +958,7 @@ CREATE TABLE TXNS (
 ) ROWDEPENDENCIES;
 
 CREATE TABLE TXN_COMPONENTS (
-  TC_TXNID NUMBER(19) REFERENCES TXNS (TXN_ID),
+  TC_TXNID NUMBER(19) NOT NULL REFERENCES TXNS (TXN_ID),
   TC_DATABASE VARCHAR2(128) NOT NULL,
   TC_TABLE VARCHAR2(128),
   TC_PARTITION VARCHAR2(767) NULL,
@@ -969,7 +969,7 @@ CREATE TABLE TXN_COMPONENTS (
 CREATE INDEX TC_TXNID_INDEX ON TXN_COMPONENTS (TC_TXNID);
 
 CREATE TABLE COMPLETED_TXN_COMPONENTS (
-  CTC_TXNID NUMBER(19),
+  CTC_TXNID NUMBER(19) NOT NULL,
   CTC_DATABASE VARCHAR2(128) NOT NULL,
   CTC_TABLE VARCHAR2(256),
   CTC_PARTITION VARCHAR2(767),
@@ -1066,26 +1066,26 @@ CREATE TABLE WRITE_SET (
 );
 
 CREATE TABLE TXN_TO_WRITE_ID (
-  T2W_TXNID number(19) NOT NULL,
-  T2W_DATABASE varchar(128) NOT NULL,
-  T2W_TABLE varchar(256) NOT NULL,
-  T2W_WRITEID number(19) NOT NULL
+  T2W_TXNID NUMBER(19) NOT NULL,
+  T2W_DATABASE VARCHAR2(128) NOT NULL,
+  T2W_TABLE VARCHAR2(256) NOT NULL,
+  T2W_WRITEID NUMBER(19) NOT NULL
 );
 
 CREATE UNIQUE INDEX TBL_TO_TXN_ID_IDX ON TXN_TO_WRITE_ID (T2W_DATABASE, T2W_TABLE, T2W_TXNID);
 CREATE UNIQUE INDEX TBL_TO_WRITE_ID_IDX ON TXN_TO_WRITE_ID (T2W_DATABASE, T2W_TABLE, T2W_WRITEID);
 
 CREATE TABLE NEXT_WRITE_ID (
-  NWI_DATABASE varchar(128) NOT NULL,
-  NWI_TABLE varchar(256) NOT NULL,
-  NWI_NEXT number(19) NOT NULL
+  NWI_DATABASE VARCHAR2(128) NOT NULL,
+  NWI_TABLE VARCHAR2(256) NOT NULL,
+  NWI_NEXT NUMBER(19) NOT NULL
 );
 
 CREATE UNIQUE INDEX NEXT_WRITE_ID_IDX ON NEXT_WRITE_ID (NWI_DATABASE, NWI_TABLE);
 
 CREATE TABLE MIN_HISTORY_LEVEL (
-  MHL_TXNID number(19) NOT NULL,
-  MHL_MIN_OPEN_TXNID number(19) NOT NULL,
+  MHL_TXNID NUMBER(19) NOT NULL,
+  MHL_MIN_OPEN_TXNID NUMBER(19) NOT NULL,
   PRIMARY KEY(MHL_TXNID)
 );
 

http://git-wip-us.apache.org/repos/asf/hive/blob/11a7164f/standalone-metastore/src/main/sql/oracle/upgrade-2.3.0-to-3.0.0.oracle.sql
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/sql/oracle/upgrade-2.3.0-to-3.0.0.oracle.sql b/standalone-metastore/src/main/sql/oracle/upgrade-2.3.0-to-3.0.0.oracle.sql
index 3be7e65..ce3437f 100644
--- a/standalone-metastore/src/main/sql/oracle/upgrade-2.3.0-to-3.0.0.oracle.sql
+++ b/standalone-metastore/src/main/sql/oracle/upgrade-2.3.0-to-3.0.0.oracle.sql
@@ -191,19 +191,19 @@ UPDATE DBS
 
 -- HIVE-18192
 CREATE TABLE TXN_TO_WRITE_ID (
-  T2W_TXNID number(19) NOT NULL,
-  T2W_DATABASE varchar(128) NOT NULL,
-  T2W_TABLE varchar(256) NOT NULL,
-  T2W_WRITEID number(19) NOT NULL
+  T2W_TXNID NUMBER(19) NOT NULL,
+  T2W_DATABASE VARCHAR2(128) NOT NULL,
+  T2W_TABLE VARCHAR2(256) NOT NULL,
+  T2W_WRITEID NUMBER(19) NOT NULL
 );
 
 CREATE UNIQUE INDEX TBL_TO_TXN_ID_IDX ON TXN_TO_WRITE_ID (T2W_DATABASE, T2W_TABLE, T2W_TXNID);
 CREATE UNIQUE INDEX TBL_TO_WRITE_ID_IDX ON TXN_TO_WRITE_ID (T2W_DATABASE, T2W_TABLE, T2W_WRITEID);
 
 CREATE TABLE NEXT_WRITE_ID (
-  NWI_DATABASE varchar(128) NOT NULL,
-  NWI_TABLE varchar(256) NOT NULL,
-  NWI_NEXT number(19) NOT NULL
+  NWI_DATABASE VARCHAR2(128) NOT NULL,
+  NWI_TABLE VARCHAR2(256) NOT NULL,
+  NWI_NEXT NUMBER(19) NOT NULL
 );
 
 CREATE UNIQUE INDEX NEXT_WRITE_ID_IDX ON NEXT_WRITE_ID (NWI_DATABASE, NWI_TABLE);
@@ -310,6 +310,31 @@ CREATE TABLE RUNTIME_STATS (
 
 CREATE INDEX IDX_RUNTIME_STATS_CREATE_TIME ON RUNTIME_STATS(CREATE_TIME);
 
+-- HIVE-18193
+-- Populate NEXT_WRITE_ID for each Transactional table and set next write ID same as next txn ID
+INSERT INTO NEXT_WRITE_ID (NWI_DATABASE, NWI_TABLE, NWI_NEXT)
+    SELECT * FROM
+        (SELECT DB.NAME, TBL_INFO.TBL_NAME FROM DBS DB,
+            (SELECT TBL.DB_ID, TBL.TBL_NAME FROM TBLS TBL,
+                (SELECT TBL_ID FROM TABLE_PARAMS WHERE PARAM_KEY='transactional' AND to_char(PARAM_VALUE)='true') TBL_PARAM
+            WHERE TBL.TBL_ID=TBL_PARAM.TBL_ID) TBL_INFO
+        where DB.DB_ID=TBL_INFO.DB_ID) DB_TBL_NAME,
+        (SELECT NTXN_NEXT FROM NEXT_TXN_ID) NEXT_WRITE;
+
+-- Populate TXN_TO_WRITE_ID for each aborted/open txns and set write ID equal to txn ID
+INSERT INTO TXN_TO_WRITE_ID (T2W_DATABASE, T2W_TABLE, T2W_TXNID, T2W_WRITEID)
+    SELECT * FROM
+        (SELECT DB.NAME, TBL_INFO.TBL_NAME FROM DBS DB,
+            (SELECT TBL.DB_ID, TBL.TBL_NAME FROM TBLS TBL,
+                (SELECT TBL_ID FROM TABLE_PARAMS WHERE PARAM_KEY='transactional' AND to_char(PARAM_VALUE)='true') TBL_PARAM
+            WHERE TBL.TBL_ID=TBL_PARAM.TBL_ID) TBL_INFO
+        where DB.DB_ID=TBL_INFO.DB_ID) DB_TBL_NAME,
+        (SELECT TXN_ID, TXN_ID as WRITE_ID FROM TXNS) TXN_INFO;
+
+-- Update TXN_COMPONENTS and COMPLETED_TXN_COMPONENTS for write ID which is same as txn ID
+UPDATE TXN_COMPONENTS SET TC_WRITEID = TC_TXNID;
+UPDATE COMPLETED_TXN_COMPONENTS SET CTC_WRITEID = CTC_TXNID;
+
 -- These lines need to be last.  Insert any changes above.
 UPDATE VERSION SET SCHEMA_VERSION='3.0.0', VERSION_COMMENT='Hive release version 3.0.0' where VER_ID=1;
 SELECT 'Finished upgrading MetaStore schema from 2.3.0 to 3.0.0' AS Status from dual;

http://git-wip-us.apache.org/repos/asf/hive/blob/11a7164f/standalone-metastore/src/main/sql/postgres/hive-schema-3.0.0.postgres.sql
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/sql/postgres/hive-schema-3.0.0.postgres.sql b/standalone-metastore/src/main/sql/postgres/hive-schema-3.0.0.postgres.sql
index 0152f48..d210a55 100644
--- a/standalone-metastore/src/main/sql/postgres/hive-schema-3.0.0.postgres.sql
+++ b/standalone-metastore/src/main/sql/postgres/hive-schema-3.0.0.postgres.sql
@@ -1645,7 +1645,7 @@ CREATE TABLE TXNS (
 );
 
 CREATE TABLE TXN_COMPONENTS (
-  TC_TXNID bigint REFERENCES TXNS (TXN_ID),
+  TC_TXNID bigint NOT NULL REFERENCES TXNS (TXN_ID),
   TC_DATABASE varchar(128) NOT NULL,
   TC_TABLE varchar(128),
   TC_PARTITION varchar(767) DEFAULT NULL,
@@ -1656,7 +1656,7 @@ CREATE TABLE TXN_COMPONENTS (
 CREATE INDEX TC_TXNID_INDEX ON TXN_COMPONENTS USING hash (TC_TXNID);
 
 CREATE TABLE COMPLETED_TXN_COMPONENTS (
-  CTC_TXNID bigint,
+  CTC_TXNID bigint NOT NULL,
   CTC_DATABASE varchar(128) NOT NULL,
   CTC_TABLE varchar(256),
   CTC_PARTITION varchar(767),

http://git-wip-us.apache.org/repos/asf/hive/blob/11a7164f/standalone-metastore/src/main/sql/postgres/upgrade-2.3.0-to-3.0.0.postgres.sql
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/sql/postgres/upgrade-2.3.0-to-3.0.0.postgres.sql b/standalone-metastore/src/main/sql/postgres/upgrade-2.3.0-to-3.0.0.postgres.sql
index fed8a93..f2571d8 100644
--- a/standalone-metastore/src/main/sql/postgres/upgrade-2.3.0-to-3.0.0.postgres.sql
+++ b/standalone-metastore/src/main/sql/postgres/upgrade-2.3.0-to-3.0.0.postgres.sql
@@ -329,6 +329,31 @@ CREATE TABLE RUNTIME_STATS (
 
 CREATE INDEX IDX_RUNTIME_STATS_CREATE_TIME ON RUNTIME_STATS(CREATE_TIME);
 
+-- HIVE-18193
+-- Populate NEXT_WRITE_ID for each Transactional table and set next write ID same as next txn ID
+INSERT INTO NEXT_WRITE_ID (NWI_DATABASE, NWI_TABLE, NWI_NEXT)
+    SELECT * FROM
+        (SELECT "DB"."NAME", "TBL_INFO"."TBL_NAME" FROM "DBS" "DB",
+            (SELECT "TBL"."DB_ID", "TBL"."TBL_NAME" FROM "TBLS" "TBL",
+                (SELECT "TBL_ID" FROM "TABLE_PARAMS" WHERE "PARAM_KEY"='transactional' AND "PARAM_VALUE"='true') "TBL_PARAM"
+            WHERE "TBL"."TBL_ID"="TBL_PARAM"."TBL_ID") "TBL_INFO"
+        where "DB"."DB_ID"="TBL_INFO"."DB_ID") "DB_TBL_NAME",
+        (SELECT NTXN_NEXT FROM NEXT_TXN_ID) "NEXT_WRITE";
+
+-- Populate TXN_TO_WRITE_ID for each aborted/open txns and set write ID equal to txn ID
+INSERT INTO TXN_TO_WRITE_ID (T2W_DATABASE, T2W_TABLE, T2W_TXNID, T2W_WRITEID)
+    SELECT * FROM
+        (SELECT "DB"."NAME", "TBL_INFO"."TBL_NAME" FROM "DBS" "DB",
+            (SELECT "TBL"."DB_ID", "TBL"."TBL_NAME" FROM "TBLS" "TBL",
+                (SELECT "TBL_ID" FROM "TABLE_PARAMS" WHERE "PARAM_KEY"='transactional' AND "PARAM_VALUE"='true') "TBL_PARAM"
+            WHERE "TBL"."TBL_ID"="TBL_PARAM"."TBL_ID") "TBL_INFO"
+        where "DB"."DB_ID"="TBL_INFO"."DB_ID") "DB_TBL_NAME",
+        (SELECT TXN_ID, TXN_ID as WRITE_ID FROM TXNS) "TXN_INFO";
+
+-- Update TXN_COMPONENTS and COMPLETED_TXN_COMPONENTS for write ID which is same as txn ID
+UPDATE TXN_COMPONENTS SET TC_WRITEID = TC_TXNID;
+UPDATE COMPLETED_TXN_COMPONENTS SET CTC_WRITEID = CTC_TXNID;
+
 -- These lines need to be last.  Insert any changes above.
 UPDATE "VERSION" SET "SCHEMA_VERSION"='3.0.0', "VERSION_COMMENT"='Hive release version 3.0.0' where "VER_ID"=1;
 SELECT 'Finished upgrading MetaStore schema from 2.3.0 to 3.0.0';


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

Posted by vg...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/e2d4f347/ql/src/test/results/clientpositive/vectorized_timestamp_funcs.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/vectorized_timestamp_funcs.q.out b/ql/src/test/results/clientpositive/vectorized_timestamp_funcs.q.out
index 244aca6..01e915b 100644
--- a/ql/src/test/results/clientpositive/vectorized_timestamp_funcs.q.out
+++ b/ql/src/test/results/clientpositive/vectorized_timestamp_funcs.q.out
@@ -1,15 +1,19 @@
-PREHOOK: query: CREATE TABLE alltypesorc_string(ctimestamp1 timestamp, stimestamp1 string) STORED AS ORC
+PREHOOK: query: CREATE TABLE alltypesorc_string(cboolean1 boolean, ctimestamp1 timestamp, stimestamp1 string,
+    ctimestamp2 timestamp) STORED AS ORC
 PREHOOK: type: CREATETABLE
 PREHOOK: Output: database:default
 PREHOOK: Output: default@alltypesorc_string
-POSTHOOK: query: CREATE TABLE alltypesorc_string(ctimestamp1 timestamp, stimestamp1 string) STORED AS ORC
+POSTHOOK: query: CREATE TABLE alltypesorc_string(cboolean1 boolean, ctimestamp1 timestamp, stimestamp1 string,
+    ctimestamp2 timestamp) STORED AS ORC
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@alltypesorc_string
 PREHOOK: query: INSERT OVERWRITE TABLE alltypesorc_string
 SELECT
+  cboolean1,
   to_utc_timestamp(ctimestamp1, 'America/Los_Angeles') AS toutc,
-  CAST(to_utc_timestamp(ctimestamp1, 'America/Los_Angeles') AS STRING) as cst
+  CAST(to_utc_timestamp(ctimestamp1, 'America/Los_Angeles') AS STRING) as cst,
+  ctimestamp2
 FROM alltypesorc
 ORDER BY toutc, cst
 LIMIT 40
@@ -18,16 +22,146 @@ PREHOOK: Input: default@alltypesorc
 PREHOOK: Output: default@alltypesorc_string
 POSTHOOK: query: INSERT OVERWRITE TABLE alltypesorc_string
 SELECT
+  cboolean1,
   to_utc_timestamp(ctimestamp1, 'America/Los_Angeles') AS toutc,
-  CAST(to_utc_timestamp(ctimestamp1, 'America/Los_Angeles') AS STRING) as cst
+  CAST(to_utc_timestamp(ctimestamp1, 'America/Los_Angeles') AS STRING) as cst,
+  ctimestamp2
 FROM alltypesorc
 ORDER BY toutc, cst
 LIMIT 40
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@alltypesorc
 POSTHOOK: Output: default@alltypesorc_string
+POSTHOOK: Lineage: alltypesorc_string.cboolean1 SIMPLE [(alltypesorc)alltypesorc.FieldSchema(name:cboolean1, type:boolean, comment:null), ]
 POSTHOOK: Lineage: alltypesorc_string.ctimestamp1 EXPRESSION [(alltypesorc)alltypesorc.FieldSchema(name:ctimestamp1, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: alltypesorc_string.ctimestamp2 SIMPLE [(alltypesorc)alltypesorc.FieldSchema(name:ctimestamp2, type:timestamp, comment:null), ]
 POSTHOOK: Lineage: alltypesorc_string.stimestamp1 EXPRESSION [(alltypesorc)alltypesorc.FieldSchema(name:ctimestamp1, type:timestamp, comment:null), ]
+PREHOOK: query: INSERT INTO TABLE alltypesorc_string values (false, '2021-09-24 03:18:32.4', '1978-08-05 14:41:05.501', '1999-10-03 16:59:10.396903939')
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@alltypesorc_string
+POSTHOOK: query: INSERT INTO TABLE alltypesorc_string values (false, '2021-09-24 03:18:32.4', '1978-08-05 14:41:05.501', '1999-10-03 16:59:10.396903939')
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@alltypesorc_string
+POSTHOOK: Lineage: alltypesorc_string.cboolean1 SCRIPT []
+POSTHOOK: Lineage: alltypesorc_string.ctimestamp1 SCRIPT []
+POSTHOOK: Lineage: alltypesorc_string.ctimestamp2 SCRIPT []
+POSTHOOK: Lineage: alltypesorc_string.stimestamp1 SCRIPT []
+PREHOOK: query: INSERT INTO TABLE alltypesorc_string values (false, null, '2013-04-10 00:43:46.8547315', null)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@alltypesorc_string
+POSTHOOK: query: INSERT INTO TABLE alltypesorc_string values (false, null, '2013-04-10 00:43:46.8547315', null)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@alltypesorc_string
+POSTHOOK: Lineage: alltypesorc_string.cboolean1 SCRIPT []
+POSTHOOK: Lineage: alltypesorc_string.ctimestamp1 EXPRESSION []
+POSTHOOK: Lineage: alltypesorc_string.ctimestamp2 EXPRESSION []
+POSTHOOK: Lineage: alltypesorc_string.stimestamp1 SCRIPT []
+PREHOOK: query: INSERT INTO TABLE alltypesorc_string values (false, '2021-09-24 03:18:32.4', null, null)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@alltypesorc_string
+POSTHOOK: query: INSERT INTO TABLE alltypesorc_string values (false, '2021-09-24 03:18:32.4', null, null)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@alltypesorc_string
+POSTHOOK: Lineage: alltypesorc_string.cboolean1 SCRIPT []
+POSTHOOK: Lineage: alltypesorc_string.ctimestamp1 SCRIPT []
+POSTHOOK: Lineage: alltypesorc_string.ctimestamp2 EXPRESSION []
+POSTHOOK: Lineage: alltypesorc_string.stimestamp1 EXPRESSION []
+PREHOOK: query: INSERT INTO TABLE alltypesorc_string values (null, '7160-12-02 06:00:24.81200852', '0004-09-22 18:26:29.519542222', '1966-08-16 13:36:50.183')
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@alltypesorc_string
+POSTHOOK: query: INSERT INTO TABLE alltypesorc_string values (null, '7160-12-02 06:00:24.81200852', '0004-09-22 18:26:29.519542222', '1966-08-16 13:36:50.183')
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@alltypesorc_string
+POSTHOOK: Lineage: alltypesorc_string.cboolean1 EXPRESSION []
+POSTHOOK: Lineage: alltypesorc_string.ctimestamp1 SCRIPT []
+POSTHOOK: Lineage: alltypesorc_string.ctimestamp2 SCRIPT []
+POSTHOOK: Lineage: alltypesorc_string.stimestamp1 SCRIPT []
+PREHOOK: query: INSERT INTO TABLE alltypesorc_string values (null, null, '4966-12-04 09:30:55.202', null)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@alltypesorc_string
+POSTHOOK: query: INSERT INTO TABLE alltypesorc_string values (null, null, '4966-12-04 09:30:55.202', null)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@alltypesorc_string
+POSTHOOK: Lineage: alltypesorc_string.cboolean1 EXPRESSION []
+POSTHOOK: Lineage: alltypesorc_string.ctimestamp1 EXPRESSION []
+POSTHOOK: Lineage: alltypesorc_string.ctimestamp2 EXPRESSION []
+POSTHOOK: Lineage: alltypesorc_string.stimestamp1 SCRIPT []
+PREHOOK: query: INSERT INTO TABLE alltypesorc_string values (null, '7160-12-02 06:00:24.81200852', null, null)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@alltypesorc_string
+POSTHOOK: query: INSERT INTO TABLE alltypesorc_string values (null, '7160-12-02 06:00:24.81200852', null, null)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@alltypesorc_string
+POSTHOOK: Lineage: alltypesorc_string.cboolean1 EXPRESSION []
+POSTHOOK: Lineage: alltypesorc_string.ctimestamp1 SCRIPT []
+POSTHOOK: Lineage: alltypesorc_string.ctimestamp2 EXPRESSION []
+POSTHOOK: Lineage: alltypesorc_string.stimestamp1 EXPRESSION []
+PREHOOK: query: INSERT INTO TABLE alltypesorc_string values (true, '1985-07-20 09:30:11.0', '8521-01-16 20:42:05.668832', '1319-02-02 16:31:57.778')
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@alltypesorc_string
+POSTHOOK: query: INSERT INTO TABLE alltypesorc_string values (true, '1985-07-20 09:30:11.0', '8521-01-16 20:42:05.668832', '1319-02-02 16:31:57.778')
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@alltypesorc_string
+POSTHOOK: Lineage: alltypesorc_string.cboolean1 SCRIPT []
+POSTHOOK: Lineage: alltypesorc_string.ctimestamp1 SCRIPT []
+POSTHOOK: Lineage: alltypesorc_string.ctimestamp2 SCRIPT []
+POSTHOOK: Lineage: alltypesorc_string.stimestamp1 SCRIPT []
+PREHOOK: query: INSERT INTO TABLE alltypesorc_string values (true, null, '1883-04-17 04:14:34.64776', '2024-11-11 16:42:41.101')
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@alltypesorc_string
+POSTHOOK: query: INSERT INTO TABLE alltypesorc_string values (true, null, '1883-04-17 04:14:34.64776', '2024-11-11 16:42:41.101')
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@alltypesorc_string
+POSTHOOK: Lineage: alltypesorc_string.cboolean1 SCRIPT []
+POSTHOOK: Lineage: alltypesorc_string.ctimestamp1 EXPRESSION []
+POSTHOOK: Lineage: alltypesorc_string.ctimestamp2 SCRIPT []
+POSTHOOK: Lineage: alltypesorc_string.stimestamp1 SCRIPT []
+PREHOOK: query: INSERT INTO TABLE alltypesorc_string values (true, '0528-10-27 08:15:18.941718273', null, null)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@alltypesorc_string
+POSTHOOK: query: INSERT INTO TABLE alltypesorc_string values (true, '0528-10-27 08:15:18.941718273', null, null)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@alltypesorc_string
+POSTHOOK: Lineage: alltypesorc_string.cboolean1 SCRIPT []
+POSTHOOK: Lineage: alltypesorc_string.ctimestamp1 SCRIPT []
+POSTHOOK: Lineage: alltypesorc_string.ctimestamp2 EXPRESSION []
+POSTHOOK: Lineage: alltypesorc_string.stimestamp1 EXPRESSION []
+PREHOOK: query: INSERT INTO TABLE alltypesorc_string values
+     (false, '2021-09-24 03:18:32.4', '1985-11-18 16:37:54.0', '2010-04-08 02:43:35.861742727'),
+     (true, null, '1985-11-18 16:37:54.0', null),
+     (null, '2021-09-24 03:18:32.4', null, '1974-10-04 17:21:03.989')
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@alltypesorc_string
+POSTHOOK: query: INSERT INTO TABLE alltypesorc_string values
+     (false, '2021-09-24 03:18:32.4', '1985-11-18 16:37:54.0', '2010-04-08 02:43:35.861742727'),
+     (true, null, '1985-11-18 16:37:54.0', null),
+     (null, '2021-09-24 03:18:32.4', null, '1974-10-04 17:21:03.989')
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@alltypesorc_string
+POSTHOOK: Lineage: alltypesorc_string.cboolean1 SCRIPT []
+POSTHOOK: Lineage: alltypesorc_string.ctimestamp1 SCRIPT []
+POSTHOOK: Lineage: alltypesorc_string.ctimestamp2 SCRIPT []
+POSTHOOK: Lineage: alltypesorc_string.stimestamp1 SCRIPT []
 PREHOOK: query: CREATE TABLE alltypesorc_wrong(stimestamp1 string) STORED AS ORC
 PREHOOK: type: CREATETABLE
 PREHOOK: Output: database:default
@@ -72,7 +206,15 @@ PREHOOK: query: EXPLAIN VECTORIZATION EXPRESSION  SELECT
   weekofyear(ctimestamp1),
   hour(ctimestamp1),
   minute(ctimestamp1),
-  second(ctimestamp1)
+  second(ctimestamp1),
+  cboolean1,
+  ctimestamp1,
+  ctimestamp2,
+  if (cboolean1, ctimestamp1, timestamp '1319-02-02 16:31:57.778'),
+  if (cboolean1, timestamp '2000-12-18 08:42:30.0005', ctimestamp1),
+  if (cboolean1, ctimestamp1, ctimestamp2),
+  if (cboolean1, ctimestamp1, null),
+  if (cboolean1, null, ctimestamp2)
 FROM alltypesorc_string
 ORDER BY c1
 PREHOOK: type: QUERY
@@ -85,7 +227,15 @@ POSTHOOK: query: EXPLAIN VECTORIZATION EXPRESSION  SELECT
   weekofyear(ctimestamp1),
   hour(ctimestamp1),
   minute(ctimestamp1),
-  second(ctimestamp1)
+  second(ctimestamp1),
+  cboolean1,
+  ctimestamp1,
+  ctimestamp2,
+  if (cboolean1, ctimestamp1, timestamp '1319-02-02 16:31:57.778'),
+  if (cboolean1, timestamp '2000-12-18 08:42:30.0005', ctimestamp1),
+  if (cboolean1, ctimestamp1, ctimestamp2),
+  if (cboolean1, ctimestamp1, null),
+  if (cboolean1, null, ctimestamp2)
 FROM alltypesorc_string
 ORDER BY c1
 POSTHOOK: type: QUERY
@@ -103,18 +253,18 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: alltypesorc_string
-            Statistics: Num rows: 40 Data size: 84 Basic stats: COMPLETE Column stats: NONE
+            Statistics: Num rows: 52 Data size: 3515 Basic stats: COMPLETE Column stats: NONE
             TableScan Vectorization:
                 native: true
             Select Operator
-              expressions: to_unix_timestamp(ctimestamp1) (type: bigint), year(ctimestamp1) (type: int), month(ctimestamp1) (type: int), day(ctimestamp1) (type: int), dayofmonth(ctimestamp1) (type: int), weekofyear(ctimestamp1) (type: int), hour(ctimestamp1) (type: int), minute(ctimestamp1) (type: int), second(ctimestamp1) (type: int)
-              outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
+              expressions: to_unix_timestamp(ctimestamp1) (type: bigint), year(ctimestamp1) (type: int), month(ctimestamp1) (type: int), day(ctimestamp1) (type: int), dayofmonth(ctimestamp1) (type: int), weekofyear(ctimestamp1) (type: int), hour(ctimestamp1) (type: int), minute(ctimestamp1) (type: int), second(ctimestamp1) (type: int), cboolean1 (type: boolean), ctimestamp1 (type: timestamp), ctimestamp2 (type: timestamp), if(cboolean1, ctimestamp1, TIMESTAMP'1319-02-02 16:31:57.778') (type: timestamp), if(cboolean1, TIMESTAMP'2000-12-18 08:42:30.0005', ctimestamp1) (type: timestamp), if(cboolean1, ctimestamp1, ctimestamp2) (type: timestamp), if(cboolean1, ctimestamp1, null) (type: timestamp), if(cboolean1, null, ctimestamp2) (type: timestamp)
+              outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16
               Select Vectorization:
                   className: VectorSelectOperator
                   native: true
-                  projectedOutputColumnNums: [3, 4, 5, 6, 7, 8, 9, 10, 11]
-                  selectExpressions: VectorUDFUnixTimeStampTimestamp(col 0:timestamp) -> 3:bigint, VectorUDFYearTimestamp(col 0:timestamp, field YEAR) -> 4:int, VectorUDFMonthTimestamp(col 0:timestamp, field MONTH) -> 5:int, VectorUDFDayOfMonthTimestamp(col 0:timestamp, field DAY_OF_MONTH) -> 6:int, VectorUDFDayOfMonthTimestamp(col 0:timestamp, field DAY_OF_MONTH) -> 7:int, VectorUDFWeekOfYearTimestamp(col 0:timestamp, field WEEK_OF_YEAR) -> 8:int, VectorUDFHourTimestamp(col 0:timestamp, field HOUR_OF_DAY) -> 9:int, VectorUDFMinuteTimestamp(col 0:timestamp, field MINUTE) -> 10:int, VectorUDFSecondTimestamp(col 0:timestamp, field SECOND) -> 11:int
-              Statistics: Num rows: 40 Data size: 84 Basic stats: COMPLETE Column stats: NONE
+                  projectedOutputColumnNums: [5, 6, 7, 8, 9, 10, 11, 12, 13, 0, 1, 3, 14, 15, 16, 17, 18]
+                  selectExpressions: VectorUDFUnixTimeStampTimestamp(col 1:timestamp) -> 5:bigint, VectorUDFYearTimestamp(col 1:timestamp, field YEAR) -> 6:int, VectorUDFMonthTimestamp(col 1:timestamp, field MONTH) -> 7:int, VectorUDFDayOfMonthTimestamp(col 1:timestamp, field DAY_OF_MONTH) -> 8:int, VectorUDFDayOfMonthTimestamp(col 1:timestamp, field DAY_OF_MONTH) -> 9:int, VectorUDFWeekOfYearTimestamp(col 1:timestamp, field WEEK_OF_YEAR) -> 10:int, VectorUDFHourTimestamp(col 1:timestamp, field HOUR_OF_DAY) -> 11:int, VectorUDFMinuteTimestamp(col 1:timestamp, field MINUTE) -> 12:int, VectorUDFSecondTimestamp(col 1:timestamp, field SECOND) -> 13:int, IfExprTimestampColumnScalar(col 0:boolean, col 1:timestamp, val 1319-02-02 16:31:57.778) -> 14:timestamp, IfExprTimestampScalarColumn(col 0:boolean, val 2000-12-18 08:42:30.0005, col 1:timestamp) -> 15:timestamp, IfExprTimestampColumnColumn(col 0:boolean, col 1:timestampcol 3:timestamp) -> 16:timestamp, IfExprColumnNull(col 0:boolean, co
 l 1:timestamp, null)(children: col 0:boolean, col 1:timestamp) -> 17:timestamp, IfExprNullColumn(col 0:boolean, null, col 3)(children: col 0:boolean, col 3:timestamp) -> 18:timestamp
+              Statistics: Num rows: 52 Data size: 3515 Basic stats: COMPLETE Column stats: NONE
               Reduce Output Operator
                 key expressions: _col0 (type: bigint)
                 sort order: +
@@ -123,8 +273,8 @@ STAGE PLANS:
                     native: false
                     nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true
                     nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false
-                Statistics: Num rows: 40 Data size: 84 Basic stats: COMPLETE Column stats: NONE
-                value expressions: _col1 (type: int), _col2 (type: int), _col3 (type: int), _col4 (type: int), _col5 (type: int), _col6 (type: int), _col7 (type: int), _col8 (type: int)
+                Statistics: Num rows: 52 Data size: 3515 Basic stats: COMPLETE Column stats: NONE
+                value expressions: _col1 (type: int), _col2 (type: int), _col3 (type: int), _col4 (type: int), _col5 (type: int), _col6 (type: int), _col7 (type: int), _col8 (type: int), _col9 (type: boolean), _col10 (type: timestamp), _col11 (type: timestamp), _col12 (type: timestamp), _col13 (type: timestamp), _col14 (type: timestamp), _col15 (type: timestamp), _col16 (type: timestamp)
       Execution mode: vectorized
       Map Vectorization:
           enabled: true
@@ -141,12 +291,12 @@ STAGE PLANS:
           enableConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false
       Reduce Operator Tree:
         Select Operator
-          expressions: KEY.reducesinkkey0 (type: bigint), VALUE._col0 (type: int), VALUE._col1 (type: int), VALUE._col2 (type: int), VALUE._col3 (type: int), VALUE._col4 (type: int), VALUE._col5 (type: int), VALUE._col6 (type: int), VALUE._col7 (type: int)
-          outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
-          Statistics: Num rows: 40 Data size: 84 Basic stats: COMPLETE Column stats: NONE
+          expressions: KEY.reducesinkkey0 (type: bigint), VALUE._col0 (type: int), VALUE._col1 (type: int), VALUE._col2 (type: int), VALUE._col3 (type: int), VALUE._col4 (type: int), VALUE._col5 (type: int), VALUE._col6 (type: int), VALUE._col7 (type: int), VALUE._col8 (type: boolean), VALUE._col9 (type: timestamp), VALUE._col10 (type: timestamp), VALUE._col11 (type: timestamp), VALUE._col12 (type: timestamp), VALUE._col13 (type: timestamp), VALUE._col14 (type: timestamp), VALUE._col15 (type: timestamp)
+          outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16
+          Statistics: Num rows: 52 Data size: 3515 Basic stats: COMPLETE Column stats: NONE
           File Output Operator
             compressed: false
-            Statistics: Num rows: 40 Data size: 84 Basic stats: COMPLETE Column stats: NONE
+            Statistics: Num rows: 52 Data size: 3515 Basic stats: COMPLETE Column stats: NONE
             table:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -167,7 +317,15 @@ PREHOOK: query: SELECT
   weekofyear(ctimestamp1),
   hour(ctimestamp1),
   minute(ctimestamp1),
-  second(ctimestamp1)
+  second(ctimestamp1),
+  cboolean1,
+  ctimestamp1,
+  ctimestamp2,
+  if (cboolean1, ctimestamp1, timestamp '1319-02-02 16:31:57.778'),
+  if (cboolean1, timestamp '2000-12-18 08:42:30.0005', ctimestamp1),
+  if (cboolean1, ctimestamp1, ctimestamp2),
+  if (cboolean1, ctimestamp1, null),
+  if (cboolean1, null, ctimestamp2)
 FROM alltypesorc_string
 ORDER BY c1
 PREHOOK: type: QUERY
@@ -182,52 +340,72 @@ POSTHOOK: query: SELECT
   weekofyear(ctimestamp1),
   hour(ctimestamp1),
   minute(ctimestamp1),
-  second(ctimestamp1)
+  second(ctimestamp1),
+  cboolean1,
+  ctimestamp1,
+  ctimestamp2,
+  if (cboolean1, ctimestamp1, timestamp '1319-02-02 16:31:57.778'),
+  if (cboolean1, timestamp '2000-12-18 08:42:30.0005', ctimestamp1),
+  if (cboolean1, ctimestamp1, ctimestamp2),
+  if (cboolean1, ctimestamp1, null),
+  if (cboolean1, null, ctimestamp2)
 FROM alltypesorc_string
 ORDER BY c1
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@alltypesorc_string
 #### A masked pattern was here ####
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
+-45479000681	528	10	27	27	43	8	15	18	true	0528-10-27 08:15:18.941718273	NULL	0528-10-27 08:15:18.941718273	2000-12-18 08:42:30.0005	0528-10-27 08:15:18.941718273	0528-10-27 08:15:18.941718273	NULL
+1632478712	2021	9	24	24	38	3	18	32	NULL	2021-09-24 03:18:32.4	1974-10-04 17:21:03.989	1319-02-02 16:31:57.778	2021-09-24 03:18:32.4	1974-10-04 17:21:03.989	NULL	1974-10-04 17:21:03.989
+1632478712	2021	9	24	24	38	3	18	32	false	2021-09-24 03:18:32.4	1999-10-03 16:59:10.396903939	1319-02-02 16:31:57.778	2021-09-24 03:18:32.4	1999-10-03 16:59:10.396903939	NULL	1999-10-03 16:59:10.396903939
+1632478712	2021	9	24	24	38	3	18	32	false	2021-09-24 03:18:32.4	2010-04-08 02:43:35.861742727	1319-02-02 16:31:57.778	2021-09-24 03:18:32.4	2010-04-08 02:43:35.861742727	NULL	2010-04-08 02:43:35.861742727
+1632478712	2021	9	24	24	38	3	18	32	false	2021-09-24 03:18:32.4	NULL	1319-02-02 16:31:57.778	2021-09-24 03:18:32.4	NULL	NULL	NULL
+163809612024	7160	12	2	2	48	6	0	24	NULL	7160-12-02 06:00:24.81200852	1966-08-16 13:36:50.183	1319-02-02 16:31:57.778	7160-12-02 06:00:24.81200852	1966-08-16 13:36:50.183	NULL	1966-08-16 13:36:50.183
+163809612024	7160	12	2	2	48	6	0	24	NULL	7160-12-02 06:00:24.81200852	NULL	1319-02-02 16:31:57.778	7160-12-02 06:00:24.81200852	NULL	NULL	NULL
+490725011	1985	7	20	20	29	9	30	11	true	1985-07-20 09:30:11	1319-02-02 16:31:57.778	1985-07-20 09:30:11	2000-12-18 08:42:30.0005	1985-07-20 09:30:11	1985-07-20 09:30:11	NULL
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	1319-02-02 16:31:57.778	NULL	NULL	NULL	NULL
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	false	NULL	1969-12-31 15:59:44.028	1319-02-02 16:31:57.778	NULL	1969-12-31 15:59:44.028	NULL	1969-12-31 15:59:44.028
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	false	NULL	1969-12-31 15:59:44.809	1319-02-02 16:31:57.778	NULL	1969-12-31 15:59:44.809	NULL	1969-12-31 15:59:44.809
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	false	NULL	1969-12-31 15:59:45.949	1319-02-02 16:31:57.778	NULL	1969-12-31 15:59:45.949	NULL	1969-12-31 15:59:45.949
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	false	NULL	1969-12-31 15:59:50.531	1319-02-02 16:31:57.778	NULL	1969-12-31 15:59:50.531	NULL	1969-12-31 15:59:50.531
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	false	NULL	1969-12-31 15:59:51.009	1319-02-02 16:31:57.778	NULL	1969-12-31 15:59:51.009	NULL	1969-12-31 15:59:51.009
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	false	NULL	1969-12-31 15:59:53.761	1319-02-02 16:31:57.778	NULL	1969-12-31 15:59:53.761	NULL	1969-12-31 15:59:53.761
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	false	NULL	1969-12-31 16:00:00.905	1319-02-02 16:31:57.778	NULL	1969-12-31 16:00:00.905	NULL	1969-12-31 16:00:00.905
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	false	NULL	1969-12-31 16:00:03.586	1319-02-02 16:31:57.778	NULL	1969-12-31 16:00:03.586	NULL	1969-12-31 16:00:03.586
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	false	NULL	1969-12-31 16:00:05.227	1319-02-02 16:31:57.778	NULL	1969-12-31 16:00:05.227	NULL	1969-12-31 16:00:05.227
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	false	NULL	1969-12-31 16:00:05.535	1319-02-02 16:31:57.778	NULL	1969-12-31 16:00:05.535	NULL	1969-12-31 16:00:05.535
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	false	NULL	1969-12-31 16:00:07.02	1319-02-02 16:31:57.778	NULL	1969-12-31 16:00:07.02	NULL	1969-12-31 16:00:07.02
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	false	NULL	1969-12-31 16:00:07.365	1319-02-02 16:31:57.778	NULL	1969-12-31 16:00:07.365	NULL	1969-12-31 16:00:07.365
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	false	NULL	1969-12-31 16:00:07.517	1319-02-02 16:31:57.778	NULL	1969-12-31 16:00:07.517	NULL	1969-12-31 16:00:07.517
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	false	NULL	1969-12-31 16:00:07.767	1319-02-02 16:31:57.778	NULL	1969-12-31 16:00:07.767	NULL	1969-12-31 16:00:07.767
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	false	NULL	1969-12-31 16:00:08.602	1319-02-02 16:31:57.778	NULL	1969-12-31 16:00:08.602	NULL	1969-12-31 16:00:08.602
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	false	NULL	1969-12-31 16:00:09.938	1319-02-02 16:31:57.778	NULL	1969-12-31 16:00:09.938	NULL	1969-12-31 16:00:09.938
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	false	NULL	1969-12-31 16:00:14.214	1319-02-02 16:31:57.778	NULL	1969-12-31 16:00:14.214	NULL	1969-12-31 16:00:14.214
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	false	NULL	1969-12-31 16:00:14.783	1319-02-02 16:31:57.778	NULL	1969-12-31 16:00:14.783	NULL	1969-12-31 16:00:14.783
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	false	NULL	NULL	1319-02-02 16:31:57.778	NULL	NULL	NULL	NULL
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	false	NULL	NULL	1319-02-02 16:31:57.778	NULL	NULL	NULL	NULL
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	true	NULL	1969-12-31 15:59:43.773	NULL	2000-12-18 08:42:30.0005	NULL	NULL	NULL
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	true	NULL	1969-12-31 15:59:44.262	NULL	2000-12-18 08:42:30.0005	NULL	NULL	NULL
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	true	NULL	1969-12-31 15:59:44.568	NULL	2000-12-18 08:42:30.0005	NULL	NULL	NULL
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	true	NULL	1969-12-31 15:59:45.697	NULL	2000-12-18 08:42:30.0005	NULL	NULL	NULL
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	true	NULL	1969-12-31 15:59:47.351	NULL	2000-12-18 08:42:30.0005	NULL	NULL	NULL
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	true	NULL	1969-12-31 15:59:47.446	NULL	2000-12-18 08:42:30.0005	NULL	NULL	NULL
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	true	NULL	1969-12-31 15:59:48.023	NULL	2000-12-18 08:42:30.0005	NULL	NULL	NULL
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	true	NULL	1969-12-31 15:59:48.629	NULL	2000-12-18 08:42:30.0005	NULL	NULL	NULL
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	true	NULL	1969-12-31 15:59:49.177	NULL	2000-12-18 08:42:30.0005	NULL	NULL	NULL
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	true	NULL	1969-12-31 15:59:49.208	NULL	2000-12-18 08:42:30.0005	NULL	NULL	NULL
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	true	NULL	1969-12-31 15:59:50.789	NULL	2000-12-18 08:42:30.0005	NULL	NULL	NULL
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	true	NULL	1969-12-31 15:59:51.245	NULL	2000-12-18 08:42:30.0005	NULL	NULL	NULL
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	true	NULL	1969-12-31 15:59:52.372	NULL	2000-12-18 08:42:30.0005	NULL	NULL	NULL
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	true	NULL	1969-12-31 15:59:55.249	NULL	2000-12-18 08:42:30.0005	NULL	NULL	NULL
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	true	NULL	1969-12-31 16:00:00.661	NULL	2000-12-18 08:42:30.0005	NULL	NULL	NULL
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	true	NULL	1969-12-31 16:00:00.784	NULL	2000-12-18 08:42:30.0005	NULL	NULL	NULL
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	true	NULL	1969-12-31 16:00:09.313	NULL	2000-12-18 08:42:30.0005	NULL	NULL	NULL
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	true	NULL	1969-12-31 16:00:09.538	NULL	2000-12-18 08:42:30.0005	NULL	NULL	NULL
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	true	NULL	1969-12-31 16:00:09.986	NULL	2000-12-18 08:42:30.0005	NULL	NULL	NULL
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	true	NULL	1969-12-31 16:00:11.031	NULL	2000-12-18 08:42:30.0005	NULL	NULL	NULL
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	true	NULL	1969-12-31 16:00:11.465	NULL	2000-12-18 08:42:30.0005	NULL	NULL	NULL
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	true	NULL	2024-11-11 16:42:41.101	NULL	2000-12-18 08:42:30.0005	NULL	NULL	NULL
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	true	NULL	NULL	NULL	2000-12-18 08:42:30.0005	NULL	NULL	NULL
 PREHOOK: query: EXPLAIN VECTORIZATION EXPRESSION  SELECT
   to_unix_timestamp(stimestamp1) AS c1,
   year(stimestamp1),
@@ -268,7 +446,7 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: alltypesorc_string
-            Statistics: Num rows: 40 Data size: 84 Basic stats: COMPLETE Column stats: NONE
+            Statistics: Num rows: 52 Data size: 3515 Basic stats: COMPLETE Column stats: NONE
             TableScan Vectorization:
                 native: true
             Select Operator
@@ -277,9 +455,9 @@ STAGE PLANS:
               Select Vectorization:
                   className: VectorSelectOperator
                   native: true
-                  projectedOutputColumnNums: [3, 4, 5, 6, 7, 8, 9, 10, 11]
-                  selectExpressions: VectorUDFUnixTimeStampString(col 1:string) -> 3:bigint, VectorUDFYearString(col 1:string, fieldStart 0, fieldLength 4) -> 4:int, VectorUDFMonthString(col 1:string, fieldStart 5, fieldLength 2) -> 5:int, VectorUDFDayOfMonthString(col 1:string, fieldStart 8, fieldLength 2) -> 6:int, VectorUDFDayOfMonthString(col 1:string, fieldStart 8, fieldLength 2) -> 7:int, VectorUDFWeekOfYearString(col 1:string) -> 8:int, VectorUDFHourString(col 1:string, fieldStart 11, fieldLength 2) -> 9:int, VectorUDFMinuteString(col 1:string, fieldStart 14, fieldLength 2) -> 10:int, VectorUDFSecondString(col 1:string, fieldStart 17, fieldLength 2) -> 11:int
-              Statistics: Num rows: 40 Data size: 84 Basic stats: COMPLETE Column stats: NONE
+                  projectedOutputColumnNums: [5, 6, 7, 8, 9, 10, 11, 12, 13]
+                  selectExpressions: VectorUDFUnixTimeStampString(col 2:string) -> 5:bigint, VectorUDFYearString(col 2:string, fieldStart 0, fieldLength 4) -> 6:int, VectorUDFMonthString(col 2:string, fieldStart 5, fieldLength 2) -> 7:int, VectorUDFDayOfMonthString(col 2:string, fieldStart 8, fieldLength 2) -> 8:int, VectorUDFDayOfMonthString(col 2:string, fieldStart 8, fieldLength 2) -> 9:int, VectorUDFWeekOfYearString(col 2:string) -> 10:int, VectorUDFHourString(col 2:string, fieldStart 11, fieldLength 2) -> 11:int, VectorUDFMinuteString(col 2:string, fieldStart 14, fieldLength 2) -> 12:int, VectorUDFSecondString(col 2:string, fieldStart 17, fieldLength 2) -> 13:int
+              Statistics: Num rows: 52 Data size: 3515 Basic stats: COMPLETE Column stats: NONE
               Reduce Output Operator
                 key expressions: _col0 (type: bigint)
                 sort order: +
@@ -288,7 +466,7 @@ STAGE PLANS:
                     native: false
                     nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true
                     nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false
-                Statistics: Num rows: 40 Data size: 84 Basic stats: COMPLETE Column stats: NONE
+                Statistics: Num rows: 52 Data size: 3515 Basic stats: COMPLETE Column stats: NONE
                 value expressions: _col1 (type: int), _col2 (type: int), _col3 (type: int), _col4 (type: int), _col5 (type: int), _col6 (type: int), _col7 (type: int), _col8 (type: int)
       Execution mode: vectorized
       Map Vectorization:
@@ -308,10 +486,10 @@ STAGE PLANS:
         Select Operator
           expressions: KEY.reducesinkkey0 (type: bigint), VALUE._col0 (type: int), VALUE._col1 (type: int), VALUE._col2 (type: int), VALUE._col3 (type: int), VALUE._col4 (type: int), VALUE._col5 (type: int), VALUE._col6 (type: int), VALUE._col7 (type: int)
           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
-          Statistics: Num rows: 40 Data size: 84 Basic stats: COMPLETE Column stats: NONE
+          Statistics: Num rows: 52 Data size: 3515 Basic stats: COMPLETE Column stats: NONE
           File Output Operator
             compressed: false
-            Statistics: Num rows: 40 Data size: 84 Basic stats: COMPLETE Column stats: NONE
+            Statistics: Num rows: 52 Data size: 3515 Basic stats: COMPLETE Column stats: NONE
             table:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -353,6 +531,18 @@ ORDER BY c1
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@alltypesorc_string
 #### A masked pattern was here ####
+-2736243926	1883	4	17	17	16	4	14	34
+-62018170411	4	9	22	22	39	18	26	29
+1365579826	2013	4	10	10	15	0	43	46
+206731024925	8521	1	16	16	3	20	42	5
+271201265	1978	8	5	5	31	14	41	5
+501208674	1985	11	18	18	47	16	37	54
+501208674	1985	11	18	18	47	16	37	54
+94573848655	4966	12	4	4	49	9	30	55
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
 NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
 NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
 NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
@@ -433,7 +623,7 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: alltypesorc_string
-            Statistics: Num rows: 40 Data size: 84 Basic stats: COMPLETE Column stats: NONE
+            Statistics: Num rows: 52 Data size: 3515 Basic stats: COMPLETE Column stats: NONE
             TableScan Vectorization:
                 native: true
             Select Operator
@@ -442,9 +632,9 @@ STAGE PLANS:
               Select Vectorization:
                   className: VectorSelectOperator
                   native: true
-                  projectedOutputColumnNums: [5, 6, 7, 8, 9, 10, 11, 12, 13]
-                  selectExpressions: LongColEqualLongColumn(col 3:bigint, col 4:bigint)(children: VectorUDFUnixTimeStampTimestamp(col 0:timestamp) -> 3:bigint, VectorUDFUnixTimeStampString(col 1:string) -> 4:bigint) -> 5:boolean, LongColEqualLongColumn(col 3:int, col 4:int)(children: VectorUDFYearTimestamp(col 0:timestamp, field YEAR) -> 3:int, VectorUDFYearString(col 1:string, fieldStart 0, fieldLength 4) -> 4:int) -> 6:boolean, LongColEqualLongColumn(col 3:int, col 4:int)(children: VectorUDFMonthTimestamp(col 0:timestamp, field MONTH) -> 3:int, VectorUDFMonthString(col 1:string, fieldStart 5, fieldLength 2) -> 4:int) -> 7:boolean, LongColEqualLongColumn(col 3:int, col 4:int)(children: VectorUDFDayOfMonthTimestamp(col 0:timestamp, field DAY_OF_MONTH) -> 3:int, VectorUDFDayOfMonthString(col 1:string, fieldStart 8, fieldLength 2) -> 4:int) -> 8:boolean, LongColEqualLongColumn(col 3:int, col 4:int)(children: VectorUDFDayOfMonthTimestamp(col 0:timestamp, field DAY_OF_MONTH) -> 3:int, V
 ectorUDFDayOfMonthString(col 1:string, fieldStart 8, fieldLength 2) -> 4:int) -> 9:boolean, LongColEqualLongColumn(col 3:int, col 4:int)(children: VectorUDFWeekOfYearTimestamp(col 0:timestamp, field WEEK_OF_YEAR) -> 3:int, VectorUDFWeekOfYearString(col 1:string) -> 4:int) -> 10:boolean, LongColEqualLongColumn(col 3:int, col 4:int)(children: VectorUDFHourTimestamp(col 0:timestamp, field HOUR_OF_DAY) -> 3:int, VectorUDFHourString(col 1:string, fieldStart 11, fieldLength 2) -> 4:int) -> 11:boolean, LongColEqualLongColumn(col 3:int, col 4:int)(children: VectorUDFMinuteTimestamp(col 0:timestamp, field MINUTE) -> 3:int, VectorUDFMinuteString(col 1:string, fieldStart 14, fieldLength 2) -> 4:int) -> 12:boolean, LongColEqualLongColumn(col 3:int, col 4:int)(children: VectorUDFSecondTimestamp(col 0:timestamp, field SECOND) -> 3:int, VectorUDFSecondString(col 1:string, fieldStart 17, fieldLength 2) -> 4:int) -> 13:boolean
-              Statistics: Num rows: 40 Data size: 84 Basic stats: COMPLETE Column stats: NONE
+                  projectedOutputColumnNums: [7, 8, 9, 10, 11, 12, 13, 14, 15]
+                  selectExpressions: LongColEqualLongColumn(col 5:bigint, col 6:bigint)(children: VectorUDFUnixTimeStampTimestamp(col 1:timestamp) -> 5:bigint, VectorUDFUnixTimeStampString(col 2:string) -> 6:bigint) -> 7:boolean, LongColEqualLongColumn(col 5:int, col 6:int)(children: VectorUDFYearTimestamp(col 1:timestamp, field YEAR) -> 5:int, VectorUDFYearString(col 2:string, fieldStart 0, fieldLength 4) -> 6:int) -> 8:boolean, LongColEqualLongColumn(col 5:int, col 6:int)(children: VectorUDFMonthTimestamp(col 1:timestamp, field MONTH) -> 5:int, VectorUDFMonthString(col 2:string, fieldStart 5, fieldLength 2) -> 6:int) -> 9:boolean, LongColEqualLongColumn(col 5:int, col 6:int)(children: VectorUDFDayOfMonthTimestamp(col 1:timestamp, field DAY_OF_MONTH) -> 5:int, VectorUDFDayOfMonthString(col 2:string, fieldStart 8, fieldLength 2) -> 6:int) -> 10:boolean, LongColEqualLongColumn(col 5:int, col 6:int)(children: VectorUDFDayOfMonthTimestamp(col 1:timestamp, field DAY_OF_MONTH) -> 5:int, 
 VectorUDFDayOfMonthString(col 2:string, fieldStart 8, fieldLength 2) -> 6:int) -> 11:boolean, LongColEqualLongColumn(col 5:int, col 6:int)(children: VectorUDFWeekOfYearTimestamp(col 1:timestamp, field WEEK_OF_YEAR) -> 5:int, VectorUDFWeekOfYearString(col 2:string) -> 6:int) -> 12:boolean, LongColEqualLongColumn(col 5:int, col 6:int)(children: VectorUDFHourTimestamp(col 1:timestamp, field HOUR_OF_DAY) -> 5:int, VectorUDFHourString(col 2:string, fieldStart 11, fieldLength 2) -> 6:int) -> 13:boolean, LongColEqualLongColumn(col 5:int, col 6:int)(children: VectorUDFMinuteTimestamp(col 1:timestamp, field MINUTE) -> 5:int, VectorUDFMinuteString(col 2:string, fieldStart 14, fieldLength 2) -> 6:int) -> 14:boolean, LongColEqualLongColumn(col 5:int, col 6:int)(children: VectorUDFSecondTimestamp(col 1:timestamp, field SECOND) -> 5:int, VectorUDFSecondString(col 2:string, fieldStart 17, fieldLength 2) -> 6:int) -> 15:boolean
+              Statistics: Num rows: 52 Data size: 3515 Basic stats: COMPLETE Column stats: NONE
               Reduce Output Operator
                 key expressions: _col0 (type: boolean)
                 sort order: +
@@ -453,7 +643,7 @@ STAGE PLANS:
                     native: false
                     nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true
                     nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false
-                Statistics: Num rows: 40 Data size: 84 Basic stats: COMPLETE Column stats: NONE
+                Statistics: Num rows: 52 Data size: 3515 Basic stats: COMPLETE Column stats: NONE
                 value expressions: _col1 (type: boolean), _col2 (type: boolean), _col3 (type: boolean), _col4 (type: boolean), _col5 (type: boolean), _col6 (type: boolean), _col7 (type: boolean), _col8 (type: boolean)
       Execution mode: vectorized
       Map Vectorization:
@@ -473,10 +663,10 @@ STAGE PLANS:
         Select Operator
           expressions: KEY.reducesinkkey0 (type: boolean), VALUE._col0 (type: boolean), VALUE._col1 (type: boolean), VALUE._col2 (type: boolean), VALUE._col3 (type: boolean), VALUE._col4 (type: boolean), VALUE._col5 (type: boolean), VALUE._col6 (type: boolean), VALUE._col7 (type: boolean)
           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
-          Statistics: Num rows: 40 Data size: 84 Basic stats: COMPLETE Column stats: NONE
+          Statistics: Num rows: 52 Data size: 3515 Basic stats: COMPLETE Column stats: NONE
           File Output Operator
             compressed: false
-            Statistics: Num rows: 40 Data size: 84 Basic stats: COMPLETE Column stats: NONE
+            Statistics: Num rows: 52 Data size: 3515 Basic stats: COMPLETE Column stats: NONE
             table:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -558,6 +748,18 @@ NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
 NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
 NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
 NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
+false	false	false	false	false	false	false	false	false
+false	false	false	false	false	false	false	false	false
+false	false	false	false	false	false	false	false	false
+false	false	false	false	false	false	false	false	false
 PREHOOK: query: EXPLAIN VECTORIZATION EXPRESSION  SELECT
   to_unix_timestamp(stimestamp1) AS c1,
   year(stimestamp1),
@@ -714,7 +916,7 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: alltypesorc_string
-            Statistics: Num rows: 40 Data size: 84 Basic stats: COMPLETE Column stats: NONE
+            Statistics: Num rows: 52 Data size: 3515 Basic stats: COMPLETE Column stats: NONE
             TableScan Vectorization:
                 native: true
             Select Operator
@@ -723,12 +925,12 @@ STAGE PLANS:
               Select Vectorization:
                   className: VectorSelectOperator
                   native: true
-                  projectedOutputColumnNums: [0]
-              Statistics: Num rows: 40 Data size: 84 Basic stats: COMPLETE Column stats: NONE
+                  projectedOutputColumnNums: [1]
+              Statistics: Num rows: 52 Data size: 3515 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
                 aggregations: min(ctimestamp1), max(ctimestamp1), count(ctimestamp1), count()
                 Group By Vectorization:
-                    aggregators: VectorUDAFMinTimestamp(col 0:timestamp) -> timestamp, VectorUDAFMaxTimestamp(col 0:timestamp) -> timestamp, VectorUDAFCount(col 0:timestamp) -> bigint, VectorUDAFCountStar(*) -> bigint
+                    aggregators: VectorUDAFMinTimestamp(col 1:timestamp) -> timestamp, VectorUDAFMaxTimestamp(col 1:timestamp) -> timestamp, VectorUDAFCount(col 1:timestamp) -> bigint, VectorUDAFCountStar(*) -> bigint
                     className: VectorGroupByOperator
                     groupByMode: HASH
                     native: false
@@ -798,7 +1000,7 @@ FROM alltypesorc_string
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@alltypesorc_string
 #### A masked pattern was here ####
-NULL	NULL	0	40
+0528-10-27 08:15:18.941718273	7160-12-02 06:00:24.81200852	8	52
 PREHOOK: query: EXPLAIN VECTORIZATION EXPRESSION  SELECT
   round(sum(ctimestamp1), 3)
 FROM alltypesorc_string
@@ -821,7 +1023,7 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: alltypesorc_string
-            Statistics: Num rows: 40 Data size: 84 Basic stats: COMPLETE Column stats: NONE
+            Statistics: Num rows: 52 Data size: 3515 Basic stats: COMPLETE Column stats: NONE
             TableScan Vectorization:
                 native: true
             Select Operator
@@ -830,12 +1032,12 @@ STAGE PLANS:
               Select Vectorization:
                   className: VectorSelectOperator
                   native: true
-                  projectedOutputColumnNums: [0]
-              Statistics: Num rows: 40 Data size: 84 Basic stats: COMPLETE Column stats: NONE
+                  projectedOutputColumnNums: [1]
+              Statistics: Num rows: 52 Data size: 3515 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
                 aggregations: sum(ctimestamp1)
                 Group By Vectorization:
-                    aggregators: VectorUDAFSumTimestamp(col 0:timestamp) -> double
+                    aggregators: VectorUDAFSumTimestamp(col 1:timestamp) -> double
                     className: VectorGroupByOperator
                     groupByMode: HASH
                     native: false
@@ -903,7 +1105,7 @@ FROM alltypesorc_string
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@alltypesorc_string
 #### A masked pattern was here ####
-NULL
+2.89160863229166E11
 PREHOOK: query: EXPLAIN VECTORIZATION EXPRESSION  SELECT
   round(avg(ctimestamp1), 0),
   variance(ctimestamp1) between 8.97077295279421E19 and 8.97077295279422E19,
@@ -940,7 +1142,7 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: alltypesorc_string
-            Statistics: Num rows: 40 Data size: 84 Basic stats: COMPLETE Column stats: NONE
+            Statistics: Num rows: 52 Data size: 3515 Basic stats: COMPLETE Column stats: NONE
             TableScan Vectorization:
                 native: true
             Select Operator
@@ -949,13 +1151,13 @@ STAGE PLANS:
               Select Vectorization:
                   className: VectorSelectOperator
                   native: true
-                  projectedOutputColumnNums: [0, 3, 6]
-                  selectExpressions: CastTimestampToDouble(col 0:timestamp) -> 3:double, DoubleColMultiplyDoubleColumn(col 4:double, col 5:double)(children: CastTimestampToDouble(col 0:timestamp) -> 4:double, CastTimestampToDouble(col 0:timestamp) -> 5:double) -> 6:double
-              Statistics: Num rows: 40 Data size: 84 Basic stats: COMPLETE Column stats: NONE
+                  projectedOutputColumnNums: [1, 5, 8]
+                  selectExpressions: CastTimestampToDouble(col 1:timestamp) -> 5:double, DoubleColMultiplyDoubleColumn(col 6:double, col 7:double)(children: CastTimestampToDouble(col 1:timestamp) -> 6:double, CastTimestampToDouble(col 1:timestamp) -> 7:double) -> 8:double
+              Statistics: Num rows: 52 Data size: 3515 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
                 aggregations: sum(_col0), count(_col0), sum(_col2), sum(_col1)
                 Group By Vectorization:
-                    aggregators: VectorUDAFSumTimestamp(col 0:timestamp) -> double, VectorUDAFCount(col 0:timestamp) -> bigint, VectorUDAFSumDouble(col 6:double) -> double, VectorUDAFSumDouble(col 3:double) -> double
+                    aggregators: VectorUDAFSumTimestamp(col 1:timestamp) -> double, VectorUDAFCount(col 1:timestamp) -> bigint, VectorUDAFSumDouble(col 8:double) -> double, VectorUDAFSumDouble(col 5:double) -> double
                     className: VectorGroupByOperator
                     groupByMode: HASH
                     native: false
@@ -1037,4 +1239,4 @@ FROM alltypesorc_string
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@alltypesorc_string
 #### A masked pattern was here ####
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
+3.6145107904E10	false	false	false	7.5245155692476E10	7.5245155692476E10	7.5245155692476E10	8.0440455033059E10

http://git-wip-us.apache.org/repos/asf/hive/blob/e2d4f347/storage-api/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizedRowBatch.java
----------------------------------------------------------------------
diff --git a/storage-api/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizedRowBatch.java b/storage-api/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizedRowBatch.java
index bebf769..f8ed7e2 100644
--- a/storage-api/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizedRowBatch.java
+++ b/storage-api/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizedRowBatch.java
@@ -180,6 +180,70 @@ public class VectorizedRowBatch implements Writable {
     return b.toString();
   }
 
+  private void appendVectorType(StringBuilder b, ColumnVector cv) {
+    String colVectorType = null;
+    if (cv instanceof LongColumnVector) {
+      colVectorType = "LONG";
+    } else if (cv instanceof DoubleColumnVector) {
+      colVectorType = "DOUBLE";
+    } else if (cv instanceof BytesColumnVector) {
+      colVectorType = "BYTES";
+    } else if (cv instanceof DecimalColumnVector) {
+      colVectorType = "DECIMAL";
+    } else if (cv instanceof TimestampColumnVector) {
+      colVectorType = "TIMESTAMP";
+    } else if (cv instanceof IntervalDayTimeColumnVector) {
+      colVectorType = "INTERVAL_DAY_TIME";
+    } else if (cv instanceof ListColumnVector) {
+      colVectorType = "LIST";
+    } else if (cv instanceof MapColumnVector) {
+      colVectorType = "MAP";
+    } else if (cv instanceof StructColumnVector) {
+      colVectorType = "STRUCT";
+    } else if (cv instanceof UnionColumnVector) {
+      colVectorType = "UNION";
+    } else {
+      colVectorType = "Unknown";
+    }
+    b.append(colVectorType);
+
+    if (cv instanceof ListColumnVector) {
+      ListColumnVector listColumnVector = (ListColumnVector) cv;
+      b.append("<");
+      appendVectorType(b, listColumnVector.child);
+      b.append(">");
+    } else if (cv instanceof MapColumnVector) {
+      MapColumnVector mapColumnVector = (MapColumnVector) cv;
+      b.append("<");
+      appendVectorType(b, mapColumnVector.keys);
+      b.append(", ");
+      appendVectorType(b, mapColumnVector.values);
+      b.append(">");
+    } else if (cv instanceof StructColumnVector) {
+      StructColumnVector structColumnVector = (StructColumnVector) cv;
+      b.append("<");
+      final int fieldCount = structColumnVector.fields.length;
+      for (int i = 0; i < fieldCount; i++) {
+        if (i > 0) {
+          b.append(", ");
+        }
+        appendVectorType(b, structColumnVector.fields[i]);
+      }
+      b.append(">");
+    } else if (cv instanceof UnionColumnVector) {
+      UnionColumnVector unionColumnVector = (UnionColumnVector) cv;
+      b.append("<");
+      final int fieldCount = unionColumnVector.fields.length;
+      for (int i = 0; i < fieldCount; i++) {
+        if (i > 0) {
+          b.append(", ");
+        }
+        appendVectorType(b, unionColumnVector.fields[i]);
+      }
+      b.append(">");
+    }
+  }
+
   public String stringify(String prefix) {
     if (size == 0) {
       return "";
@@ -195,33 +259,10 @@ public class VectorizedRowBatch implements Writable {
       }
       b.append(projIndex);
       b.append(":");
-      String colVectorType = null;
-      if (cv instanceof LongColumnVector) {
-        colVectorType = "LONG";
-      } else if (cv instanceof DoubleColumnVector) {
-        colVectorType = "DOUBLE";
-      } else if (cv instanceof BytesColumnVector) {
-        colVectorType = "BYTES";
-      } else if (cv instanceof DecimalColumnVector) {
-        colVectorType = "DECIMAL";
-      } else if (cv instanceof TimestampColumnVector) {
-        colVectorType = "TIMESTAMP";
-      } else if (cv instanceof IntervalDayTimeColumnVector) {
-        colVectorType = "INTERVAL_DAY_TIME";
-      } else if (cv instanceof ListColumnVector) {
-        colVectorType = "LIST";
-      } else if (cv instanceof MapColumnVector) {
-        colVectorType = "MAP";
-      } else if (cv instanceof StructColumnVector) {
-        colVectorType = "STRUCT";
-      } else if (cv instanceof UnionColumnVector) {
-        colVectorType = "UNION";
-      } else {
-        colVectorType = "Unknown";
-      }
-      b.append(colVectorType);
+      appendVectorType(b, cv);
     }
     b.append('\n');
+    b.append(prefix);
 
     if (this.selectedInUse) {
       for (int j = 0; j < size; j++) {

http://git-wip-us.apache.org/repos/asf/hive/blob/e2d4f347/storage-api/src/test/org/apache/hadoop/hive/ql/exec/vector/TestStructColumnVector.java
----------------------------------------------------------------------
diff --git a/storage-api/src/test/org/apache/hadoop/hive/ql/exec/vector/TestStructColumnVector.java b/storage-api/src/test/org/apache/hadoop/hive/ql/exec/vector/TestStructColumnVector.java
index 7bc03ed..3d9f262 100644
--- a/storage-api/src/test/org/apache/hadoop/hive/ql/exec/vector/TestStructColumnVector.java
+++ b/storage-api/src/test/org/apache/hadoop/hive/ql/exec/vector/TestStructColumnVector.java
@@ -119,7 +119,7 @@ public class TestStructColumnVector {
       byte[] buffer = ("value " + r).getBytes(StandardCharsets.UTF_8);
       y.setRef(r, buffer, 0, buffer.length);
     }
-    final String EXPECTED = ("Column vector types: 0:STRUCT, 1:BYTES\n" +
+    final String EXPECTED = ("Column vector types: 0:STRUCT<LONG, TIMESTAMP>, 1:BYTES\n" +
         "[[0, 2000-01-01 00:00:01.0], \"value 0\"]\n" +
         "[[3, 2000-01-01 00:00:02.0], \"value 1\"]\n" +
         "[[6, 2000-01-01 00:00:03.0], \"value 2\"]\n" +
@@ -153,7 +153,7 @@ public class TestStructColumnVector {
       byte[] buffer = ("value " + r).getBytes(StandardCharsets.UTF_8);
       y.setRef(r, buffer, 0, buffer.length);
     }
-    final String EXPECTED = ("Column vector types: 0:STRUCT, 1:BYTES\n" +
+    final String EXPECTED = ("Column vector types: 0:STRUCT<LONG, TIMESTAMP>, 1:BYTES\n" +
         "[[0, 2000-01-01 00:00:01], \"value 0\"]\n" +
         "[[3, 2000-01-01 00:00:02], \"value 1\"]\n" +
         "[[6, 2000-01-01 00:00:03], \"value 2\"]\n" +

http://git-wip-us.apache.org/repos/asf/hive/blob/e2d4f347/vector-code-gen/src/org/apache/hadoop/hive/tools/GenVectorCode.java
----------------------------------------------------------------------
diff --git a/vector-code-gen/src/org/apache/hadoop/hive/tools/GenVectorCode.java b/vector-code-gen/src/org/apache/hadoop/hive/tools/GenVectorCode.java
index fbb89a9..b5220a0 100644
--- a/vector-code-gen/src/org/apache/hadoop/hive/tools/GenVectorCode.java
+++ b/vector-code-gen/src/org/apache/hadoop/hive/tools/GenVectorCode.java
@@ -1063,6 +1063,15 @@ public class GenVectorCode extends Task {
       {"IfExprScalarScalar", "long", "double"},
       {"IfExprScalarScalar", "double", "double"},
 
+      {"IfExprObjectColumnColumn", "timestamp"},
+      {"IfExprObjectColumnColumn", "interval_day_time"},
+      {"IfExprObjectColumnScalar", "timestamp"},
+      {"IfExprObjectColumnScalar", "interval_day_time"},
+      {"IfExprObjectScalarColumn", "timestamp"},
+      {"IfExprObjectScalarColumn", "interval_day_time"},
+      {"IfExprObjectScalarScalar", "timestamp"},
+      {"IfExprObjectScalarScalar", "interval_day_time"},
+
       // template, <ClassName>, <ValueType>, <OperatorSymbol>, <DescriptionName>, <DescriptionValue>
       {"VectorUDAFMinMax", "VectorUDAFMinLong", "long", "<", "min",
           "_FUNC_(expr) - Returns the minimum value of expr (vectorized, type: long)"},
@@ -1385,6 +1394,12 @@ public class GenVectorCode extends Task {
         generateIfExprScalarColumn(tdesc);
       } else if (tdesc[0].equals("IfExprScalarScalar")) {
         generateIfExprScalarScalar(tdesc);
+      } else if (
+          tdesc[0].equals("IfExprObjectColumnColumn") ||
+          tdesc[0].equals("IfExprObjectColumnScalar") ||
+          tdesc[0].equals("IfExprObjectScalarColumn") ||
+          tdesc[0].equals("IfExprObjectScalarScalar")) {
+        generateIfExprObject(tdesc);
       } else if (tdesc[0].equals("FilterDecimalColumnCompareDecimalScalar")) {
         generateFilterDecimalColumnCompareDecimalScalar(tdesc);
       } else if (tdesc[0].equals("FilterDecimalScalarCompareDecimalColumn")) {
@@ -2259,6 +2274,46 @@ public class GenVectorCode extends Task {
         className, templateString);
   }
 
+  private void generateIfExprObject(String [] tdesc) throws Exception {
+    String typeName = tdesc[1];
+    String objectName;
+    String scalarType;
+    String scalarImport;
+    if (typeName.equals("timestamp")) {
+      objectName = "Timestamp";
+      scalarType = "Timestamp";
+      scalarImport = "java.sql.Timestamp";
+    } else if (typeName.equals("interval_day_time")) {
+      objectName = "IntervalDayTime";
+      scalarType = "HiveIntervalDayTime";
+      scalarImport = "org.apache.hadoop.hive.common.type.HiveIntervalDayTime";
+    } else {
+      objectName = "unknown";
+      scalarType = "unknown";
+      scalarImport = "unknown";
+    }
+    String classNameSuffix = tdesc[0].substring("IfExprObject".length());
+
+    String writableType = getOutputWritableType(typeName);
+    String columnVectorType = getColumnVectorType(typeName);
+
+    String className = "IfExpr" + objectName + classNameSuffix;
+
+    File templateFile = new File(joinPath(this.expressionTemplateDirectory, tdesc[0] + ".txt"));
+    String templateString = readFile(templateFile);
+
+    templateString = templateString.replaceAll("<ClassName>", className);
+    templateString = templateString.replaceAll("<ScalarType>", scalarType);
+    templateString = templateString.replaceAll("<ScalarImport>", scalarImport);
+    templateString = templateString.replaceAll("<TypeName>", typeName);
+    templateString = templateString.replaceAll("<ObjectName>", objectName);
+    templateString = templateString.replaceAll("<WritableType>", writableType);
+    templateString = templateString.replaceAll("<ColumnVectorType>", columnVectorType);
+
+    writeFile(templateFile.lastModified(), expressionOutputDirectory, expressionClassesDirectory,
+        className, templateString);
+  }
+
   // template, <ClassNamePrefix>, <ReturnType>, <FuncName>
   private void generateDecimalColumnUnaryFunc(String [] tdesc) throws Exception {
     String classNamePrefix = tdesc[1];


[41/50] [abbrv] hive git commit: HIVE-19474 : Decimal type should be casted as part of the CTAS or INSERT Clause (addendum)

Posted by vg...@apache.org.
HIVE-19474 : Decimal type should be casted as part of the CTAS or INSERT Clause (addendum)


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

Branch: refs/heads/branch-3.0.0
Commit: 3e04cc117e90c3e778d59cf96cccb1b4bbc6435f
Parents: a9105ce
Author: Jesus Camacho Rodriguez <jc...@apache.org>
Authored: Mon May 14 14:44:51 2018 -0700
Committer: Vineet Garg <vg...@apache.org>
Committed: Mon May 14 14:54:23 2018 -0700

----------------------------------------------------------------------
 .../hadoop/hive/druid/serde/TestDruidSerDe.java | 24 ++++++++------------
 1 file changed, 10 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/3e04cc11/druid-handler/src/test/org/apache/hadoop/hive/druid/serde/TestDruidSerDe.java
----------------------------------------------------------------------
diff --git a/druid-handler/src/test/org/apache/hadoop/hive/druid/serde/TestDruidSerDe.java b/druid-handler/src/test/org/apache/hadoop/hive/druid/serde/TestDruidSerDe.java
index e4fa1a2..e45de0f 100644
--- a/druid-handler/src/test/org/apache/hadoop/hive/druid/serde/TestDruidSerDe.java
+++ b/druid-handler/src/test/org/apache/hadoop/hive/druid/serde/TestDruidSerDe.java
@@ -764,8 +764,8 @@ public class TestDruidSerDe {
   }
 
 
-  private static final String COLUMN_NAMES = "__time,c0,c1,c2,c3,c4,c5,c6,c7,c8,c9";
-  private static final String COLUMN_TYPES = "timestamp with local time zone,string,char(6),varchar(8),double,float,decimal(38,18),bigint,int,smallint,tinyint";
+  private static final String COLUMN_NAMES = "__time,c0,c1,c2,c3,c4,c5,c6,c7,c8";
+  private static final String COLUMN_TYPES = "timestamp with local time zone,string,char(6),varchar(8),double,float,bigint,int,smallint,tinyint";
   private static final Object[] ROW_OBJECT = new Object[] {
       new TimestampLocalTZWritable(new TimestampTZ(Instant.ofEpochMilli(1377907200000L).atZone(ZoneOffset.UTC))),
       new Text("dim1_val"),
@@ -773,7 +773,6 @@ public class TestDruidSerDe {
       new HiveVarcharWritable(new HiveVarchar("dim3_val", 8)),
       new DoubleWritable(10669.3D),
       new FloatWritable(10669.45F),
-      new HiveDecimalWritable(HiveDecimal.create(1064.34D)),
       new LongWritable(1113939),
       new IntWritable(1112123),
       new ShortWritable((short) 12),
@@ -788,11 +787,10 @@ public class TestDruidSerDe {
           .put("c2", "dim3_val")
           .put("c3", 10669.3D)
           .put("c4", 10669.45F)
-          .put("c5", 1064.34D)
-          .put("c6", 1113939L)
-          .put("c7", 1112123)
-          .put("c8", (short) 12)
-          .put("c9", (byte) 0)
+          .put("c5", 1113939L)
+          .put("c6", 1112123)
+          .put("c7", (short) 12)
+          .put("c8", (byte) 0)
           .put("__time_granularity", 1377907200000L)
           .build());
 
@@ -877,7 +875,6 @@ public class TestDruidSerDe {
       new HiveVarcharWritable(new HiveVarchar("dim3_val", 8)),
       new DoubleWritable(10669.3D),
       new FloatWritable(10669.45F),
-      new HiveDecimalWritable(HiveDecimal.create(1064.34D)),
       new LongWritable(1113939),
       new IntWritable(1112123),
       new ShortWritable((short) 12),
@@ -891,11 +888,10 @@ public class TestDruidSerDe {
           .put("c2", "dim3_val")
           .put("c3", 10669.3D)
           .put("c4", 10669.45F)
-          .put("c5", 1064.34D)
-          .put("c6", 1113939L)
-          .put("c7", 1112123)
-          .put("c8", (short) 12)
-          .put("c9", (byte) 0)
+          .put("c5", 1113939L)
+          .put("c6", 1112123)
+          .put("c7", (short) 12)
+          .put("c8", (byte) 0)
           .build());
 
   @Test


[11/50] [abbrv] hive git commit: HIVE-19448: Vectorization: sysdb test doesn't work after enabling vectorization by default (Matt McCline, reviewed by Deepak Jaiswal)

Posted by vg...@apache.org.
HIVE-19448: Vectorization: sysdb test doesn't work after enabling vectorization by default (Matt McCline, reviewed by Deepak Jaiswal)


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

Branch: refs/heads/branch-3.0.0
Commit: 78e6bfac030ab2137483dc2b97a3f58fc03c41bd
Parents: 92bc9cf
Author: Matt McCline <mm...@hortonworks.com>
Authored: Wed May 9 11:06:56 2018 -0500
Committer: Matt McCline <mm...@hortonworks.com>
Committed: Wed May 9 11:07:39 2018 -0500

----------------------------------------------------------------------
 .../org/apache/hadoop/hive/conf/HiveConf.java   |   2 +-
 .../hive/ql/exec/vector/VectorMapOperator.java  |  11 +-
 ql/src/test/queries/clientpositive/sysdb.q      |   5 +
 .../clientpositive/llap/jdbc_handler.q.out      |   2 +-
 .../results/clientpositive/llap/sysdb.q.out     | 526 ++++++++++++-------
 5 files changed, 359 insertions(+), 187 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/78e6bfac/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
----------------------------------------------------------------------
diff --git a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
index 7b0fabe..3bb1e80 100644
--- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
+++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
@@ -3530,7 +3530,7 @@ public class HiveConf extends Configuration {
         "The default value is false."),
     HIVE_VECTORIZATION_ROW_DESERIALIZE_INPUTFORMAT_EXCLUDES(
         "hive.vectorized.row.serde.inputformat.excludes",
-        "org.apache.parquet.hadoop.ParquetInputFormat,org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat",
+        "org.apache.parquet.hadoop.ParquetInputFormat,org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat,org.apache.hive.storage.jdbc.JdbcInputFormat",
         "The input formats not supported by row deserialize vectorization."),
     HIVE_VECTOR_ADAPTOR_USAGE_MODE("hive.vectorized.adaptor.usage.mode", "all", new StringSet("none", "chosen", "all"),
         "Specifies the extent to which the VectorUDFAdaptor will be used for UDFs that do not have a corresponding vectorized class.\n" +

http://git-wip-us.apache.org/repos/asf/hive/blob/78e6bfac/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorMapOperator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorMapOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorMapOperator.java
index 6f1346d..2542e03 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorMapOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorMapOperator.java
@@ -945,8 +945,15 @@ public class VectorMapOperator extends AbstractMapOperator {
 
               // Convert input row to standard objects.
               List<Object> standardObjects = new ArrayList<Object>();
-              ObjectInspectorUtils.copyToStandardObject(standardObjects, deserialized,
-                  currentPartRawRowObjectInspector, ObjectInspectorCopyOption.WRITABLE);
+              try {
+                ObjectInspectorUtils.copyToStandardObject(
+                    standardObjects,
+                    deserialized,
+                    currentPartRawRowObjectInspector,
+                    ObjectInspectorCopyOption.WRITABLE);
+              } catch (Exception e) {
+                throw new HiveException("copyToStandardObject failed: " + e);
+              }
               if (standardObjects.size() < currentDataColumnCount) {
                 throw new HiveException("Input File Format returned row with too few columns");
               }

http://git-wip-us.apache.org/repos/asf/hive/blob/78e6bfac/ql/src/test/queries/clientpositive/sysdb.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/sysdb.q b/ql/src/test/queries/clientpositive/sysdb.q
index 399c3ce..7f88fe1 100644
--- a/ql/src/test/queries/clientpositive/sysdb.q
+++ b/ql/src/test/queries/clientpositive/sysdb.q
@@ -7,6 +7,8 @@ set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager;
 
 set hive.cbo.enable=false;
 
+-- SORT_QUERY_RESULTS
+
 create table src_buck (key int, value string) clustered by(value) into 2 buckets;
 
 create table src_skew (key int) skewed by (key) on (1,2,3);
@@ -59,6 +61,9 @@ select role_name from roles order by role_name limit 5;
 
 select principal_name, grantor from role_map order by principal_name, grantor limit 5;
 
+explain vectorization detail
+select count(*) from sds;
+
 select count(*) from sds;
 
 select param_key, param_value from sd_params order by param_key, param_value limit 5;

http://git-wip-us.apache.org/repos/asf/hive/blob/78e6bfac/ql/src/test/results/clientpositive/llap/jdbc_handler.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/jdbc_handler.q.out b/ql/src/test/results/clientpositive/llap/jdbc_handler.q.out
index 94fadf1..90d88df 100644
--- a/ql/src/test/results/clientpositive/llap/jdbc_handler.q.out
+++ b/ql/src/test/results/clientpositive/llap/jdbc_handler.q.out
@@ -253,7 +253,7 @@ STAGE PLANS:
                       sort order: 
                       Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE
                       value expressions: _col0 (type: string)
-            Execution mode: vectorized, llap
+            Execution mode: llap
             LLAP IO: no inputs
         Map 4 
             Map Operator Tree:

http://git-wip-us.apache.org/repos/asf/hive/blob/78e6bfac/ql/src/test/results/clientpositive/llap/sysdb.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/sysdb.q.out b/ql/src/test/results/clientpositive/llap/sysdb.q.out
index de20f2d..3164637 100644
--- a/ql/src/test/results/clientpositive/llap/sysdb.q.out
+++ b/ql/src/test/results/clientpositive/llap/sysdb.q.out
@@ -273,7 +273,7 @@ TBLPROPERTIES (
   \"OWNER_NAME\",
   \"OWNER_TYPE\"
 FROM
-  DBS"
+  \"DBS\""
 )
 PREHOOK: type: CREATETABLE
 PREHOOK: Output: SYS@DBS
@@ -297,7 +297,7 @@ TBLPROPERTIES (
   \"OWNER_NAME\",
   \"OWNER_TYPE\"
 FROM
-  DBS"
+  \"DBS\""
 )
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: SYS@DBS
@@ -1370,7 +1370,7 @@ TBLPROPERTIES (
   \"VIEW_EXPANDED_TEXT\",
   \"VIEW_ORIGINAL_TEXT\",
   \"IS_REWRITE_ENABLED\"
-FROM TBLS"
+FROM \"TBLS\""
 )
 PREHOOK: type: CREATETABLE
 PREHOOK: Output: SYS@TBLS
@@ -1407,7 +1407,7 @@ TBLPROPERTIES (
   \"VIEW_EXPANDED_TEXT\",
   \"VIEW_ORIGINAL_TEXT\",
   \"IS_REWRITE_ENABLED\"
-FROM TBLS"
+FROM \"TBLS\""
 )
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: SYS@TBLS
@@ -1428,7 +1428,7 @@ TBLPROPERTIES (
   \"DB_NAME\",
   \"TBL_NAME\",
   \"TXN_LIST\"
-FROM MV_CREATION_METADATA"
+FROM \"MV_CREATION_METADATA\""
 )
 PREHOOK: type: CREATETABLE
 PREHOOK: Output: SYS@MV_CREATION_METADATA
@@ -1449,7 +1449,7 @@ TBLPROPERTIES (
   \"DB_NAME\",
   \"TBL_NAME\",
   \"TXN_LIST\"
-FROM MV_CREATION_METADATA"
+FROM \"MV_CREATION_METADATA\""
 )
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: SYS@MV_CREATION_METADATA
@@ -1465,7 +1465,7 @@ TBLPROPERTIES (
 "SELECT
   \"MV_CREATION_METADATA_ID\",
   \"TBL_ID\"
-FROM MV_TABLES_USED"
+FROM \"MV_TABLES_USED\""
 )
 PREHOOK: type: CREATETABLE
 PREHOOK: Output: SYS@MV_TABLES_USED
@@ -1481,7 +1481,7 @@ TBLPROPERTIES (
 "SELECT
   \"MV_CREATION_METADATA_ID\",
   \"TBL_ID\"
-FROM MV_TABLES_USED"
+FROM \"MV_TABLES_USED\""
 )
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: SYS@MV_TABLES_USED
@@ -2164,16 +2164,16 @@ TBLPROPERTIES (
 "hive.sql.database.type" = "METASTORE",
 "hive.sql.query" =
 "SELECT
-  r.NAME RP_NAME,
-  t.NAME NAME,
-  TRIGGER_EXPRESSION,
-  ACTION_EXPRESSION
+  r.\"NAME\" AS RP_NAME,
+  t.\"NAME\" AS NAME,
+  \"TRIGGER_EXPRESSION\",
+  \"ACTION_EXPRESSION\"
 FROM
-  WM_TRIGGER t
+  \"WM_TRIGGER\" t
 JOIN
-  WM_RESOURCEPLAN r
+  \"WM_RESOURCEPLAN\" r
 ON
-  t.RP_ID = r.RP_ID"
+  t.\"RP_ID\" = r.\"RP_ID\""
 )
 PREHOOK: type: CREATETABLE
 PREHOOK: Output: SYS@WM_TRIGGERS
@@ -2189,16 +2189,16 @@ TBLPROPERTIES (
 "hive.sql.database.type" = "METASTORE",
 "hive.sql.query" =
 "SELECT
-  r.NAME RP_NAME,
-  t.NAME NAME,
-  TRIGGER_EXPRESSION,
-  ACTION_EXPRESSION
+  r.\"NAME\" AS RP_NAME,
+  t.\"NAME\" AS NAME,
+  \"TRIGGER_EXPRESSION\",
+  \"ACTION_EXPRESSION\"
 FROM
-  WM_TRIGGER t
+  \"WM_TRIGGER\" t
 JOIN
-  WM_RESOURCEPLAN r
+  \"WM_RESOURCEPLAN\" r
 ON
-  t.RP_ID = r.RP_ID"
+  t.\"RP_ID\" = r.\"RP_ID\""
 )
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: SYS@WM_TRIGGERS
@@ -2215,17 +2215,17 @@ TBLPROPERTIES (
 "hive.sql.database.type" = "METASTORE",
 "hive.sql.query" =
 "SELECT
-  WM_RESOURCEPLAN.NAME,
-  WM_POOL.PATH,
-  WM_POOL.ALLOC_FRACTION,
-  WM_POOL.QUERY_PARALLELISM,
-  WM_POOL.SCHEDULING_POLICY
+  \"WM_RESOURCEPLAN\".\"NAME\",
+  \"WM_POOL\".\"PATH\",
+  \"WM_POOL\".\"ALLOC_FRACTION\",
+  \"WM_POOL\".\"QUERY_PARALLELISM\",
+  \"WM_POOL\".\"SCHEDULING_POLICY\"
 FROM
-  WM_POOL
+  \"WM_POOL\"
 JOIN
-  WM_RESOURCEPLAN
+  \"WM_RESOURCEPLAN\"
 ON
-  WM_POOL.RP_ID = WM_RESOURCEPLAN.RP_ID"
+  \"WM_POOL\".\"RP_ID\" = \"WM_RESOURCEPLAN\".\"RP_ID\""
 )
 PREHOOK: type: CREATETABLE
 PREHOOK: Output: SYS@WM_POOLS
@@ -2242,17 +2242,17 @@ TBLPROPERTIES (
 "hive.sql.database.type" = "METASTORE",
 "hive.sql.query" =
 "SELECT
-  WM_RESOURCEPLAN.NAME,
-  WM_POOL.PATH,
-  WM_POOL.ALLOC_FRACTION,
-  WM_POOL.QUERY_PARALLELISM,
-  WM_POOL.SCHEDULING_POLICY
+  \"WM_RESOURCEPLAN\".\"NAME\",
+  \"WM_POOL\".\"PATH\",
+  \"WM_POOL\".\"ALLOC_FRACTION\",
+  \"WM_POOL\".\"QUERY_PARALLELISM\",
+  \"WM_POOL\".\"SCHEDULING_POLICY\"
 FROM
-  WM_POOL
+  \"WM_POOL\"
 JOIN
-  WM_RESOURCEPLAN
+  \"WM_RESOURCEPLAN\"
 ON
-  WM_POOL.RP_ID = WM_RESOURCEPLAN.RP_ID"
+  \"WM_POOL\".\"RP_ID\" = \"WM_RESOURCEPLAN\".\"RP_ID\""
 )
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: SYS@WM_POOLS
@@ -2267,21 +2267,21 @@ TBLPROPERTIES (
 "hive.sql.database.type" = "METASTORE",
 "hive.sql.query" =
 "SELECT
-  WM_RESOURCEPLAN.NAME RP_NAME,
-  WM_POOL.PATH POOL_PATH,
-  WM_TRIGGER.NAME TRIGGER_NAME
-FROM WM_POOL_TO_TRIGGER
-  JOIN WM_POOL ON WM_POOL_TO_TRIGGER.POOL_ID = WM_POOL.POOL_ID
-  JOIN WM_TRIGGER ON WM_POOL_TO_TRIGGER.TRIGGER_ID = WM_TRIGGER.TRIGGER_ID
-  JOIN WM_RESOURCEPLAN ON WM_POOL.RP_ID = WM_RESOURCEPLAN.RP_ID
+  \"WM_RESOURCEPLAN\".\"NAME\" AS RP_NAME,
+  \"WM_POOL\".\"PATH\" AS POOL_PATH,
+  \"WM_TRIGGER\".\"NAME\" AS TRIGGER_NAME
+FROM \"WM_POOL_TO_TRIGGER\"
+  JOIN \"WM_POOL\" ON \"WM_POOL_TO_TRIGGER\".\"POOL_ID\" = \"WM_POOL\".\"POOL_ID\"
+  JOIN \"WM_TRIGGER\" ON \"WM_POOL_TO_TRIGGER\".\"TRIGGER_ID\" = \"WM_TRIGGER\".\"TRIGGER_ID\"
+  JOIN \"WM_RESOURCEPLAN\" ON \"WM_POOL\".\"RP_ID\" = \"WM_RESOURCEPLAN\".\"RP_ID\"
 UNION
 SELECT
-  WM_RESOURCEPLAN.NAME RP_NAME,
-  '<unmanaged queries>' POOL_PATH,
-  WM_TRIGGER.NAME TRIGGER_NAME
-FROM WM_TRIGGER
-  JOIN WM_RESOURCEPLAN ON WM_TRIGGER.RP_ID = WM_RESOURCEPLAN.RP_ID
-WHERE WM_TRIGGER.IS_IN_UNMANAGED = 1
+  \"WM_RESOURCEPLAN\".\"NAME\" AS RP_NAME,
+  '<unmanaged queries>' AS POOL_PATH,
+  \"WM_TRIGGER\".\"NAME\" AS TRIGGER_NAME
+FROM \"WM_TRIGGER\"
+  JOIN \"WM_RESOURCEPLAN\" ON \"WM_TRIGGER\".\"RP_ID\" = \"WM_RESOURCEPLAN\".\"RP_ID\"
+WHERE CAST(\"WM_TRIGGER\".\"IS_IN_UNMANAGED\" AS CHAR) IN ('1', 't')
 "
 )
 PREHOOK: type: CREATETABLE
@@ -2297,21 +2297,21 @@ TBLPROPERTIES (
 "hive.sql.database.type" = "METASTORE",
 "hive.sql.query" =
 "SELECT
-  WM_RESOURCEPLAN.NAME RP_NAME,
-  WM_POOL.PATH POOL_PATH,
-  WM_TRIGGER.NAME TRIGGER_NAME
-FROM WM_POOL_TO_TRIGGER
-  JOIN WM_POOL ON WM_POOL_TO_TRIGGER.POOL_ID = WM_POOL.POOL_ID
-  JOIN WM_TRIGGER ON WM_POOL_TO_TRIGGER.TRIGGER_ID = WM_TRIGGER.TRIGGER_ID
-  JOIN WM_RESOURCEPLAN ON WM_POOL.RP_ID = WM_RESOURCEPLAN.RP_ID
+  \"WM_RESOURCEPLAN\".\"NAME\" AS RP_NAME,
+  \"WM_POOL\".\"PATH\" AS POOL_PATH,
+  \"WM_TRIGGER\".\"NAME\" AS TRIGGER_NAME
+FROM \"WM_POOL_TO_TRIGGER\"
+  JOIN \"WM_POOL\" ON \"WM_POOL_TO_TRIGGER\".\"POOL_ID\" = \"WM_POOL\".\"POOL_ID\"
+  JOIN \"WM_TRIGGER\" ON \"WM_POOL_TO_TRIGGER\".\"TRIGGER_ID\" = \"WM_TRIGGER\".\"TRIGGER_ID\"
+  JOIN \"WM_RESOURCEPLAN\" ON \"WM_POOL\".\"RP_ID\" = \"WM_RESOURCEPLAN\".\"RP_ID\"
 UNION
 SELECT
-  WM_RESOURCEPLAN.NAME RP_NAME,
-  '<unmanaged queries>' POOL_PATH,
-  WM_TRIGGER.NAME TRIGGER_NAME
-FROM WM_TRIGGER
-  JOIN WM_RESOURCEPLAN ON WM_TRIGGER.RP_ID = WM_RESOURCEPLAN.RP_ID
-WHERE WM_TRIGGER.IS_IN_UNMANAGED = 1
+  \"WM_RESOURCEPLAN\".\"NAME\" AS RP_NAME,
+  '<unmanaged queries>' AS POOL_PATH,
+  \"WM_TRIGGER\".\"NAME\" AS TRIGGER_NAME
+FROM \"WM_TRIGGER\"
+  JOIN \"WM_RESOURCEPLAN\" ON \"WM_TRIGGER\".\"RP_ID\" = \"WM_RESOURCEPLAN\".\"RP_ID\"
+WHERE CAST(\"WM_TRIGGER\".\"IS_IN_UNMANAGED\" AS CHAR) IN ('1', 't')
 "
 )
 POSTHOOK: type: CREATETABLE
@@ -2329,14 +2329,14 @@ TBLPROPERTIES (
 "hive.sql.database.type" = "METASTORE",
 "hive.sql.query" =
 "SELECT
-  WM_RESOURCEPLAN.NAME,
-  ENTITY_TYPE,
-  ENTITY_NAME,
-  case when WM_POOL.PATH is null then '<unmanaged>' else WM_POOL.PATH end,
-  ORDERING
-FROM WM_MAPPING
-JOIN WM_RESOURCEPLAN ON WM_MAPPING.RP_ID = WM_RESOURCEPLAN.RP_ID
-LEFT OUTER JOIN WM_POOL ON WM_POOL.POOL_ID = WM_MAPPING.POOL_ID
+  \"WM_RESOURCEPLAN\".\"NAME\",
+  \"ENTITY_TYPE\",
+  \"ENTITY_NAME\",
+  case when \"WM_POOL\".\"PATH\" is null then '<unmanaged>' else \"WM_POOL\".\"PATH\" end,
+  \"ORDERING\"
+FROM \"WM_MAPPING\"
+JOIN \"WM_RESOURCEPLAN\" ON \"WM_MAPPING\".\"RP_ID\" = \"WM_RESOURCEPLAN\".\"RP_ID\"
+LEFT OUTER JOIN \"WM_POOL\" ON \"WM_POOL\".\"POOL_ID\" = \"WM_MAPPING\".\"POOL_ID\"
 "
 )
 PREHOOK: type: CREATETABLE
@@ -2354,14 +2354,14 @@ TBLPROPERTIES (
 "hive.sql.database.type" = "METASTORE",
 "hive.sql.query" =
 "SELECT
-  WM_RESOURCEPLAN.NAME,
-  ENTITY_TYPE,
-  ENTITY_NAME,
-  case when WM_POOL.PATH is null then '<unmanaged>' else WM_POOL.PATH end,
-  ORDERING
-FROM WM_MAPPING
-JOIN WM_RESOURCEPLAN ON WM_MAPPING.RP_ID = WM_RESOURCEPLAN.RP_ID
-LEFT OUTER JOIN WM_POOL ON WM_POOL.POOL_ID = WM_MAPPING.POOL_ID
+  \"WM_RESOURCEPLAN\".\"NAME\",
+  \"ENTITY_TYPE\",
+  \"ENTITY_NAME\",
+  case when \"WM_POOL\".\"PATH\" is null then '<unmanaged>' else \"WM_POOL\".\"PATH\" end,
+  \"ORDERING\"
+FROM \"WM_MAPPING\"
+JOIN \"WM_RESOURCEPLAN\" ON \"WM_MAPPING\".\"RP_ID\" = \"WM_RESOURCEPLAN\".\"RP_ID\"
+LEFT OUTER JOIN \"WM_POOL\" ON \"WM_POOL\".\"POOL_ID\" = \"WM_MAPPING\".\"POOL_ID\"
 "
 )
 POSTHOOK: type: CREATETABLE
@@ -2393,18 +2393,26 @@ PREHOOK: query: CREATE VIEW IF NOT EXISTS `SCHEMATA`
   `DEFAULT_CHARACTER_SET_NAME`,
   `SQL_PATH`
 ) AS
-SELECT
+SELECT DISTINCT
   'default',
-  `NAME`,
-  `OWNER_NAME`,
+  D.`NAME`,
+  D.`OWNER_NAME`,
   cast(null as string),
   cast(null as string),
   cast(null as string),
   `DB_LOCATION_URI`
 FROM
-  sys.DBS
+  `sys`.`DBS` D, `sys`.`TBLS` T, `sys`.`TBL_PRIVS` P
+WHERE
+  NOT restrict_information_schema() OR
+  D.`DB_ID` = T.`DB_ID`
+  AND T.`TBL_ID` = P.`TBL_ID`
+  AND (P.`PRINCIPAL_NAME`=current_user() AND P.`PRINCIPAL_TYPE`='USER'
+    OR ((array_contains(current_groups(), P.`PRINCIPAL_NAME`) OR P.`PRINCIPAL_NAME` = 'public') AND P.`PRINCIPAL_TYPE`='GROUP'))
 PREHOOK: type: CREATEVIEW
 PREHOOK: Input: sys@dbs
+PREHOOK: Input: sys@tbl_privs
+PREHOOK: Input: sys@tbls
 PREHOOK: Output: INFORMATION_SCHEMA@SCHEMATA
 PREHOOK: Output: database:information_schema
 POSTHOOK: query: CREATE VIEW IF NOT EXISTS `SCHEMATA`
@@ -2417,27 +2425,35 @@ POSTHOOK: query: CREATE VIEW IF NOT EXISTS `SCHEMATA`
   `DEFAULT_CHARACTER_SET_NAME`,
   `SQL_PATH`
 ) AS
-SELECT
+SELECT DISTINCT
   'default',
-  `NAME`,
-  `OWNER_NAME`,
+  D.`NAME`,
+  D.`OWNER_NAME`,
   cast(null as string),
   cast(null as string),
   cast(null as string),
   `DB_LOCATION_URI`
 FROM
-  sys.DBS
+  `sys`.`DBS` D, `sys`.`TBLS` T, `sys`.`TBL_PRIVS` P
+WHERE
+  NOT restrict_information_schema() OR
+  D.`DB_ID` = T.`DB_ID`
+  AND T.`TBL_ID` = P.`TBL_ID`
+  AND (P.`PRINCIPAL_NAME`=current_user() AND P.`PRINCIPAL_TYPE`='USER'
+    OR ((array_contains(current_groups(), P.`PRINCIPAL_NAME`) OR P.`PRINCIPAL_NAME` = 'public') AND P.`PRINCIPAL_TYPE`='GROUP'))
 POSTHOOK: type: CREATEVIEW
 POSTHOOK: Input: sys@dbs
+POSTHOOK: Input: sys@tbl_privs
+POSTHOOK: Input: sys@tbls
 POSTHOOK: Output: INFORMATION_SCHEMA@SCHEMATA
 POSTHOOK: Output: database:information_schema
 POSTHOOK: Lineage: SCHEMATA.catalog_name SIMPLE []
 POSTHOOK: Lineage: SCHEMATA.default_character_set_catalog EXPRESSION []
 POSTHOOK: Lineage: SCHEMATA.default_character_set_name EXPRESSION []
 POSTHOOK: Lineage: SCHEMATA.default_character_set_schema EXPRESSION []
-POSTHOOK: Lineage: SCHEMATA.schema_name SIMPLE [(dbs)dbs.FieldSchema(name:name, type:string, comment:from deserializer), ]
+POSTHOOK: Lineage: SCHEMATA.schema_name SIMPLE [(dbs)d.FieldSchema(name:name, type:string, comment:from deserializer), ]
 #### A masked pattern was here ####
-POSTHOOK: Lineage: SCHEMATA.sql_path SIMPLE [(dbs)dbs.FieldSchema(name:db_location_uri, type:string, comment:from deserializer), ]
+POSTHOOK: Lineage: SCHEMATA.sql_path SIMPLE [(dbs)d.FieldSchema(name:db_location_uri, type:string, comment:from deserializer), ]
 PREHOOK: query: CREATE VIEW IF NOT EXISTS `TABLES`
 (
   `TABLE_CATALOG`,
@@ -2453,7 +2469,7 @@ PREHOOK: query: CREATE VIEW IF NOT EXISTS `TABLES`
   `IS_TYPED`,
   `COMMIT_ACTION`
 ) AS
-SELECT
+SELECT DISTINCT
   'default',
   D.NAME,
   T.TBL_NAME,
@@ -2467,11 +2483,16 @@ SELECT
   'NO',
   cast(null as string)
 FROM
-  `sys`.`TBLS` T, `sys`.`DBS` D
+  `sys`.`TBLS` T, `sys`.`DBS` D, `sys`.`TBL_PRIVS` P
 WHERE
   D.`DB_ID` = T.`DB_ID`
+  AND (NOT restrict_information_schema() OR T.`TBL_ID` = P.`TBL_ID`
+  AND (P.`PRINCIPAL_NAME`=current_user() AND P.`PRINCIPAL_TYPE`='USER'
+    OR ((array_contains(current_groups(), P.`PRINCIPAL_NAME`) OR P.`PRINCIPAL_NAME` = 'public') AND P.`PRINCIPAL_TYPE`='GROUP'))
+  AND P.`TBL_PRIV`='SELECT')
 PREHOOK: type: CREATEVIEW
 PREHOOK: Input: sys@dbs
+PREHOOK: Input: sys@tbl_privs
 PREHOOK: Input: sys@tbls
 PREHOOK: Output: INFORMATION_SCHEMA@TABLES
 PREHOOK: Output: database:information_schema
@@ -2490,7 +2511,7 @@ POSTHOOK: query: CREATE VIEW IF NOT EXISTS `TABLES`
   `IS_TYPED`,
   `COMMIT_ACTION`
 ) AS
-SELECT
+SELECT DISTINCT
   'default',
   D.NAME,
   T.TBL_NAME,
@@ -2504,11 +2525,16 @@ SELECT
   'NO',
   cast(null as string)
 FROM
-  `sys`.`TBLS` T, `sys`.`DBS` D
+  `sys`.`TBLS` T, `sys`.`DBS` D, `sys`.`TBL_PRIVS` P
 WHERE
   D.`DB_ID` = T.`DB_ID`
+  AND (NOT restrict_information_schema() OR T.`TBL_ID` = P.`TBL_ID`
+  AND (P.`PRINCIPAL_NAME`=current_user() AND P.`PRINCIPAL_TYPE`='USER'
+    OR ((array_contains(current_groups(), P.`PRINCIPAL_NAME`) OR P.`PRINCIPAL_NAME` = 'public') AND P.`PRINCIPAL_TYPE`='GROUP'))
+  AND P.`TBL_PRIV`='SELECT')
 POSTHOOK: type: CREATEVIEW
 POSTHOOK: Input: sys@dbs
+POSTHOOK: Input: sys@tbl_privs
 POSTHOOK: Input: sys@tbls
 POSTHOOK: Output: INFORMATION_SCHEMA@TABLES
 POSTHOOK: Output: database:information_schema
@@ -2535,9 +2561,9 @@ PREHOOK: query: CREATE VIEW IF NOT EXISTS `TABLE_PRIVILEGES`
   `IS_GRANTABLE`,
   `WITH_HIERARCHY`
 ) AS
-SELECT
-  `GRANTOR`,
-  `PRINCIPAL_NAME`,
+SELECT DISTINCT
+  P.`GRANTOR`,
+  P.`PRINCIPAL_NAME`,
   'default',
   D.`NAME`,
   T.`TBL_NAME`,
@@ -2545,12 +2571,18 @@ SELECT
   IF (P.`GRANT_OPTION` == 0, 'NO', 'YES'),
   'NO'
 FROM
-  sys.`TBL_PRIVS` P,
-  sys.`TBLS` T,
-  sys.`DBS` D
+  `sys`.`TBL_PRIVS` P,
+  `sys`.`TBLS` T,
+  `sys`.`DBS` D,
+  `sys`.`TBL_PRIVS` P2
 WHERE
-  P.TBL_ID = T.TBL_ID
-  AND T.DB_ID = D.DB_ID
+  P.`TBL_ID` = T.`TBL_ID`
+  AND T.`DB_ID` = D.`DB_ID`
+  AND (NOT restrict_information_schema() OR
+  P.`TBL_ID` = P2.`TBL_ID` AND P.`PRINCIPAL_NAME` = P2.`PRINCIPAL_NAME` AND P.`PRINCIPAL_TYPE` = P2.`PRINCIPAL_TYPE`
+  AND (P2.`PRINCIPAL_NAME`=current_user() AND P2.`PRINCIPAL_TYPE`='USER'
+    OR ((array_contains(current_groups(), P2.`PRINCIPAL_NAME`) OR P2.`PRINCIPAL_NAME` = 'public') AND P2.`PRINCIPAL_TYPE`='GROUP'))
+  AND P2.`TBL_PRIV`='SELECT')
 PREHOOK: type: CREATEVIEW
 PREHOOK: Input: sys@dbs
 PREHOOK: Input: sys@tbl_privs
@@ -2568,9 +2600,9 @@ POSTHOOK: query: CREATE VIEW IF NOT EXISTS `TABLE_PRIVILEGES`
   `IS_GRANTABLE`,
   `WITH_HIERARCHY`
 ) AS
-SELECT
-  `GRANTOR`,
-  `PRINCIPAL_NAME`,
+SELECT DISTINCT
+  P.`GRANTOR`,
+  P.`PRINCIPAL_NAME`,
   'default',
   D.`NAME`,
   T.`TBL_NAME`,
@@ -2578,12 +2610,18 @@ SELECT
   IF (P.`GRANT_OPTION` == 0, 'NO', 'YES'),
   'NO'
 FROM
-  sys.`TBL_PRIVS` P,
-  sys.`TBLS` T,
-  sys.`DBS` D
+  `sys`.`TBL_PRIVS` P,
+  `sys`.`TBLS` T,
+  `sys`.`DBS` D,
+  `sys`.`TBL_PRIVS` P2
 WHERE
-  P.TBL_ID = T.TBL_ID
-  AND T.DB_ID = D.DB_ID
+  P.`TBL_ID` = T.`TBL_ID`
+  AND T.`DB_ID` = D.`DB_ID`
+  AND (NOT restrict_information_schema() OR
+  P.`TBL_ID` = P2.`TBL_ID` AND P.`PRINCIPAL_NAME` = P2.`PRINCIPAL_NAME` AND P.`PRINCIPAL_TYPE` = P2.`PRINCIPAL_TYPE`
+  AND (P2.`PRINCIPAL_NAME`=current_user() AND P2.`PRINCIPAL_TYPE`='USER'
+    OR ((array_contains(current_groups(), P2.`PRINCIPAL_NAME`) OR P2.`PRINCIPAL_NAME` = 'public') AND P2.`PRINCIPAL_TYPE`='GROUP'))
+  AND P2.`TBL_PRIV`='SELECT')
 POSTHOOK: type: CREATEVIEW
 POSTHOOK: Input: sys@dbs
 POSTHOOK: Input: sys@tbl_privs
@@ -2648,7 +2686,7 @@ PREHOOK: query: CREATE VIEW IF NOT EXISTS `COLUMNS`
   `DECLARED_NUMERIC_PRECISION`,
   `DECLARED_NUMERIC_SCALE`
 ) AS
-SELECT
+SELECT DISTINCT
   'default',
   D.NAME,
   T.TBL_NAME,
@@ -2737,18 +2775,26 @@ SELECT
        WHEN lower(C.TYPE_NAME) like 'numeric%' THEN 10
        ELSE null END
 FROM
-  sys.`COLUMNS_V2` C,
-  sys.`SDS` S,
-  sys.`TBLS` T,
-  sys.`DBS` D
+  `sys`.`COLUMNS_V2` C,
+  `sys`.`SDS` S,
+  `sys`.`TBLS` T,
+  `sys`.`DBS` D,
+  `sys`.`TBL_COL_PRIVS` P
 WHERE
   S.`SD_ID` = T.`SD_ID`
   AND T.`DB_ID` = D.`DB_ID`
   AND C.`CD_ID` = S.`CD_ID`
+  AND (NOT restrict_information_schema() OR
+  T.`TBL_ID` = P.`TBL_ID`
+  AND C.`COLUMN_NAME` = P.`COLUMN_NAME`
+  AND (P.`PRINCIPAL_NAME`=current_user() AND P.`PRINCIPAL_TYPE`='USER'
+    OR ((array_contains(current_groups(), P.`PRINCIPAL_NAME`) OR P.`PRINCIPAL_NAME` = 'public') AND P.`PRINCIPAL_TYPE`='GROUP'))
+  AND P.`TBL_COL_PRIV`='SELECT')
 PREHOOK: type: CREATEVIEW
 PREHOOK: Input: sys@columns_v2
 PREHOOK: Input: sys@dbs
 PREHOOK: Input: sys@sds
+PREHOOK: Input: sys@tbl_col_privs
 PREHOOK: Input: sys@tbls
 PREHOOK: Output: INFORMATION_SCHEMA@COLUMNS
 PREHOOK: Output: database:information_schema
@@ -2802,7 +2848,7 @@ POSTHOOK: query: CREATE VIEW IF NOT EXISTS `COLUMNS`
   `DECLARED_NUMERIC_PRECISION`,
   `DECLARED_NUMERIC_SCALE`
 ) AS
-SELECT
+SELECT DISTINCT
   'default',
   D.NAME,
   T.TBL_NAME,
@@ -2891,18 +2937,26 @@ SELECT
        WHEN lower(C.TYPE_NAME) like 'numeric%' THEN 10
        ELSE null END
 FROM
-  sys.`COLUMNS_V2` C,
-  sys.`SDS` S,
-  sys.`TBLS` T,
-  sys.`DBS` D
+  `sys`.`COLUMNS_V2` C,
+  `sys`.`SDS` S,
+  `sys`.`TBLS` T,
+  `sys`.`DBS` D,
+  `sys`.`TBL_COL_PRIVS` P
 WHERE
   S.`SD_ID` = T.`SD_ID`
   AND T.`DB_ID` = D.`DB_ID`
   AND C.`CD_ID` = S.`CD_ID`
+  AND (NOT restrict_information_schema() OR
+  T.`TBL_ID` = P.`TBL_ID`
+  AND C.`COLUMN_NAME` = P.`COLUMN_NAME`
+  AND (P.`PRINCIPAL_NAME`=current_user() AND P.`PRINCIPAL_TYPE`='USER'
+    OR ((array_contains(current_groups(), P.`PRINCIPAL_NAME`) OR P.`PRINCIPAL_NAME` = 'public') AND P.`PRINCIPAL_TYPE`='GROUP'))
+  AND P.`TBL_COL_PRIV`='SELECT')
 POSTHOOK: type: CREATEVIEW
 POSTHOOK: Input: sys@columns_v2
 POSTHOOK: Input: sys@dbs
 POSTHOOK: Input: sys@sds
+POSTHOOK: Input: sys@tbl_col_privs
 POSTHOOK: Input: sys@tbls
 POSTHOOK: Output: INFORMATION_SCHEMA@COLUMNS
 POSTHOOK: Output: database:information_schema
@@ -2964,32 +3018,35 @@ PREHOOK: query: CREATE VIEW IF NOT EXISTS `COLUMN_PRIVILEGES`
   `PRIVILEGE_TYPE`,
   `IS_GRANTABLE`
 ) AS
-SELECT
-  `GRANTOR`,
-  `PRINCIPAL_NAME`,
+SELECT DISTINCT
+  P.`GRANTOR`,
+  P.`PRINCIPAL_NAME`,
   'default',
   D.`NAME`,
   T.`TBL_NAME`,
-  C.`COLUMN_NAME`,
+  P.`COLUMN_NAME`,
   P.`TBL_COL_PRIV`,
   IF (P.`GRANT_OPTION` == 0, 'NO', 'YES')
 FROM
-  sys.`TBL_COL_PRIVS` P,
-  sys.`TBLS` T,
-  sys.`DBS` D,
-  sys.`COLUMNS_V2` C,
-  sys.`SDS` S
+  `sys`.`TBL_COL_PRIVS` P,
+  `sys`.`TBLS` T,
+  `sys`.`DBS` D,
+  `sys`.`SDS` S,
+  `sys`.`TBL_PRIVS` P2
 WHERE
   S.`SD_ID` = T.`SD_ID`
   AND T.`DB_ID` = D.`DB_ID`
   AND P.`TBL_ID` = T.`TBL_ID`
-  AND P.`COLUMN_NAME` = C.`COLUMN_NAME`
-  AND C.`CD_ID` = S.`CD_ID`
+  AND (NOT restrict_information_schema() OR
+  P.`TBL_ID` = P2.`TBL_ID` AND P.`PRINCIPAL_NAME` = P2.`PRINCIPAL_NAME` AND P.`PRINCIPAL_TYPE` = P2.`PRINCIPAL_TYPE`
+  AND (P2.`PRINCIPAL_NAME`=current_user() AND P2.`PRINCIPAL_TYPE`='USER'
+    OR ((array_contains(current_groups(), P2.`PRINCIPAL_NAME`) OR P2.`PRINCIPAL_NAME` = 'public') AND P2.`PRINCIPAL_TYPE`='GROUP'))
+  AND P2.`TBL_PRIV`='SELECT')
 PREHOOK: type: CREATEVIEW
-PREHOOK: Input: sys@columns_v2
 PREHOOK: Input: sys@dbs
 PREHOOK: Input: sys@sds
 PREHOOK: Input: sys@tbl_col_privs
+PREHOOK: Input: sys@tbl_privs
 PREHOOK: Input: sys@tbls
 PREHOOK: Output: INFORMATION_SCHEMA@COLUMN_PRIVILEGES
 PREHOOK: Output: database:information_schema
@@ -3004,36 +3061,39 @@ POSTHOOK: query: CREATE VIEW IF NOT EXISTS `COLUMN_PRIVILEGES`
   `PRIVILEGE_TYPE`,
   `IS_GRANTABLE`
 ) AS
-SELECT
-  `GRANTOR`,
-  `PRINCIPAL_NAME`,
+SELECT DISTINCT
+  P.`GRANTOR`,
+  P.`PRINCIPAL_NAME`,
   'default',
   D.`NAME`,
   T.`TBL_NAME`,
-  C.`COLUMN_NAME`,
+  P.`COLUMN_NAME`,
   P.`TBL_COL_PRIV`,
   IF (P.`GRANT_OPTION` == 0, 'NO', 'YES')
 FROM
-  sys.`TBL_COL_PRIVS` P,
-  sys.`TBLS` T,
-  sys.`DBS` D,
-  sys.`COLUMNS_V2` C,
-  sys.`SDS` S
+  `sys`.`TBL_COL_PRIVS` P,
+  `sys`.`TBLS` T,
+  `sys`.`DBS` D,
+  `sys`.`SDS` S,
+  `sys`.`TBL_PRIVS` P2
 WHERE
   S.`SD_ID` = T.`SD_ID`
   AND T.`DB_ID` = D.`DB_ID`
   AND P.`TBL_ID` = T.`TBL_ID`
-  AND P.`COLUMN_NAME` = C.`COLUMN_NAME`
-  AND C.`CD_ID` = S.`CD_ID`
+  AND (NOT restrict_information_schema() OR
+  P.`TBL_ID` = P2.`TBL_ID` AND P.`PRINCIPAL_NAME` = P2.`PRINCIPAL_NAME` AND P.`PRINCIPAL_TYPE` = P2.`PRINCIPAL_TYPE`
+  AND (P2.`PRINCIPAL_NAME`=current_user() AND P2.`PRINCIPAL_TYPE`='USER'
+    OR ((array_contains(current_groups(), P2.`PRINCIPAL_NAME`) OR P2.`PRINCIPAL_NAME` = 'public') AND P2.`PRINCIPAL_TYPE`='GROUP'))
+  AND P2.`TBL_PRIV`='SELECT')
 POSTHOOK: type: CREATEVIEW
-POSTHOOK: Input: sys@columns_v2
 POSTHOOK: Input: sys@dbs
 POSTHOOK: Input: sys@sds
 POSTHOOK: Input: sys@tbl_col_privs
+POSTHOOK: Input: sys@tbl_privs
 POSTHOOK: Input: sys@tbls
 POSTHOOK: Output: INFORMATION_SCHEMA@COLUMN_PRIVILEGES
 POSTHOOK: Output: database:information_schema
-POSTHOOK: Lineage: COLUMN_PRIVILEGES.column_name SIMPLE [(columns_v2)c.FieldSchema(name:column_name, type:string, comment:from deserializer), ]
+POSTHOOK: Lineage: COLUMN_PRIVILEGES.column_name SIMPLE [(tbl_col_privs)p.FieldSchema(name:column_name, type:string, comment:from deserializer), ]
 POSTHOOK: Lineage: COLUMN_PRIVILEGES.grantee SIMPLE [(tbl_col_privs)p.FieldSchema(name:principal_name, type:string, comment:from deserializer), ]
 POSTHOOK: Lineage: COLUMN_PRIVILEGES.grantor SIMPLE [(tbl_col_privs)p.FieldSchema(name:grantor, type:string, comment:from deserializer), ]
 POSTHOOK: Lineage: COLUMN_PRIVILEGES.is_grantable EXPRESSION [(tbl_col_privs)p.FieldSchema(name:grant_option, type:int, comment:from deserializer), ]
@@ -3054,7 +3114,7 @@ PREHOOK: query: CREATE VIEW IF NOT EXISTS `VIEWS`
   `IS_TRIGGER_DELETABLE`,
   `IS_TRIGGER_INSERTABLE_INTO`
 ) AS
-SELECT
+SELECT DISTINCT
   'default',
   D.NAME,
   T.TBL_NAME,
@@ -3067,12 +3127,19 @@ SELECT
   false
 FROM
   `sys`.`DBS` D,
-  `sys`.`TBLS` T
+  `sys`.`TBLS` T,
+  `sys`.`TBL_PRIVS` P
 WHERE
-   D.`DB_ID` = T.`DB_ID` AND
-   length(T.VIEW_ORIGINAL_TEXT) > 0
+  D.`DB_ID` = T.`DB_ID`
+  AND length(T.VIEW_ORIGINAL_TEXT) > 0
+  AND (NOT restrict_information_schema() OR
+  T.`TBL_ID` = P.`TBL_ID`
+  AND (P.`PRINCIPAL_NAME`=current_user() AND P.`PRINCIPAL_TYPE`='USER'
+    OR ((array_contains(current_groups(), P.`PRINCIPAL_NAME`) OR P.`PRINCIPAL_NAME` = 'public') AND P.`PRINCIPAL_TYPE`='GROUP'))
+  AND P.`TBL_PRIV`='SELECT')
 PREHOOK: type: CREATEVIEW
 PREHOOK: Input: sys@dbs
+PREHOOK: Input: sys@tbl_privs
 PREHOOK: Input: sys@tbls
 PREHOOK: Output: INFORMATION_SCHEMA@VIEWS
 PREHOOK: Output: database:information_schema
@@ -3089,7 +3156,7 @@ POSTHOOK: query: CREATE VIEW IF NOT EXISTS `VIEWS`
   `IS_TRIGGER_DELETABLE`,
   `IS_TRIGGER_INSERTABLE_INTO`
 ) AS
-SELECT
+SELECT DISTINCT
   'default',
   D.NAME,
   T.TBL_NAME,
@@ -3102,12 +3169,19 @@ SELECT
   false
 FROM
   `sys`.`DBS` D,
-  `sys`.`TBLS` T
+  `sys`.`TBLS` T,
+  `sys`.`TBL_PRIVS` P
 WHERE
-   D.`DB_ID` = T.`DB_ID` AND
-   length(T.VIEW_ORIGINAL_TEXT) > 0
+  D.`DB_ID` = T.`DB_ID`
+  AND length(T.VIEW_ORIGINAL_TEXT) > 0
+  AND (NOT restrict_information_schema() OR
+  T.`TBL_ID` = P.`TBL_ID`
+  AND (P.`PRINCIPAL_NAME`=current_user() AND P.`PRINCIPAL_TYPE`='USER'
+    OR ((array_contains(current_groups(), P.`PRINCIPAL_NAME`) OR P.`PRINCIPAL_NAME` = 'public') AND P.`PRINCIPAL_TYPE`='GROUP'))
+  AND P.`TBL_PRIV`='SELECT')
 POSTHOOK: type: CREATEVIEW
 POSTHOOK: Input: sys@dbs
+POSTHOOK: Input: sys@tbl_privs
 POSTHOOK: Input: sys@tbls
 POSTHOOK: Output: INFORMATION_SCHEMA@VIEWS
 POSTHOOK: Output: database:information_schema
@@ -3256,6 +3330,95 @@ POSTHOOK: query: select principal_name, grantor from role_map order by principal
 POSTHOOK: type: QUERY
 POSTHOOK: Input: sys@role_map
 #### A masked pattern was here ####
+PREHOOK: query: explain vectorization detail
+select count(*) from sds
+PREHOOK: type: QUERY
+POSTHOOK: query: explain vectorization detail
+select count(*) from sds
+POSTHOOK: type: QUERY
+PLAN VECTORIZATION:
+  enabled: true
+  enabledConditionsMet: [hive.vectorized.execution.enabled IS true]
+
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Edges:
+        Reducer 2 <- Map 1 (CUSTOM_SIMPLE_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: sds
+                  Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                  Select Operator
+                    Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                    Group By Operator
+                      aggregations: count()
+                      mode: hash
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: COMPLETE
+                      Reduce Output Operator
+                        sort order: 
+                        Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: COMPLETE
+                        value expressions: _col0 (type: bigint)
+            Execution mode: llap
+            LLAP IO: no inputs
+            Map Vectorization:
+                enabled: false
+                enabledConditionsNotMet: hive.vectorized.use.row.serde.deserialize IS true AND hive.vectorized.row.serde.inputformat.excludes NOT CONTAINS org.apache.hive.storage.jdbc.JdbcInputFormat IS false, hive.vectorized.use.row.serde.deserialize IS false
+                inputFileFormats: org.apache.hive.storage.jdbc.JdbcInputFormat
+        Reducer 2 
+            Execution mode: vectorized, llap
+            Reduce Vectorization:
+                enabled: true
+                enableConditionsMet: hive.vectorized.execution.reduce.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true
+                reduceColumnNullOrder: 
+                reduceColumnSortOrder: 
+                allNative: false
+                usesVectorUDFAdaptor: false
+                vectorized: true
+                rowBatchContext:
+                    dataColumnCount: 1
+                    dataColumns: VALUE._col0:bigint
+                    partitionColumnCount: 0
+                    scratchColumnTypeNames: []
+            Reduce Operator Tree:
+              Group By Operator
+                aggregations: count(VALUE._col0)
+                Group By Vectorization:
+                    aggregators: VectorUDAFCountMerge(col 0:bigint) -> bigint
+                    className: VectorGroupByOperator
+                    groupByMode: MERGEPARTIAL
+                    native: false
+                    vectorProcessingMode: GLOBAL
+                    projectedOutputColumnNums: [0]
+                mode: mergepartial
+                outputColumnNames: _col0
+                Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: COMPLETE
+                File Output Operator
+                  compressed: false
+                  File Sink Vectorization:
+                      className: VectorFileSinkOperator
+                      native: false
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: COMPLETE
+                  table:
+                      input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
 PREHOOK: query: select count(*) from sds
 PREHOOK: type: QUERY
 PREHOOK: Input: sys@sds
@@ -3283,9 +3446,9 @@ POSTHOOK: Input: sys@sequence_table
 #### A masked pattern was here ####
 org.apache.hadoop.hive.metastore.model.MCatalog
 org.apache.hadoop.hive.metastore.model.MColumnDescriptor
-org.apache.hadoop.hive.metastore.model.MCreationMetadata
 org.apache.hadoop.hive.metastore.model.MDatabase
 org.apache.hadoop.hive.metastore.model.MFunction
+org.apache.hadoop.hive.metastore.model.MGlobalPrivilege
 PREHOOK: query: select name, slib from serdes order by name, slib limit 5
 PREHOOK: type: QUERY
 PREHOOK: Input: sys@serdes
@@ -3421,8 +3584,8 @@ POSTHOOK: type: QUERY
 POSTHOOK: Input: sys@tab_col_stats
 #### A masked pattern was here ####
 alltypesorc	cbigint	3115	6064
-alltypesorc	cboolean1	3114	NULL
-alltypesorc	cboolean2	3115	NULL
+alltypesorc	cboolean1	3114	0
+alltypesorc	cboolean2	3115	0
 alltypesorc	cdouble	3114	5569
 alltypesorc	cfloat	3115	131
 alltypesorc	cint	3115	6029
@@ -3479,11 +3642,11 @@ POSTHOOK: type: QUERY
 POSTHOOK: Input: sys@table_params
 POSTHOOK: Input: sys@table_stats_view
 #### A masked pattern was here ####
-{"BASIC_STATS":"true","COLUMN_STATS":{"key":"true","value":"true"}}	0	0	0	0
-{"BASIC_STATS":"true","COLUMN_STATS":{"pool_path":"true","rp_name":"true","trigger_name":"true"}}	0	0	0	0
-{"BASIC_STATS":"true","COLUMN_STATS":{"key":"true","value":"true"}}	0	0	0	0
-{"BASIC_STATS":"true","COLUMN_STATS":{"a":"true","b":"true","c":"true","d":"true","e":"true","f":"true","g":"true"}}	0	0	0	0
-{"BASIC_STATS":"true","COLUMN_STATS":{"create_time":"true","last_access_time":"true","part_id":"true","part_name":"true","sd_id":"true","tbl_id":"true"}}	0	0	0	0
+{"BASIC_STATS":"true","COLUMN_STATS":{"action_expression":"true","name":"true","rp_name":"true","trigger_expression":"true"}}	0	0	0	0
+#### A masked pattern was here ####
+{"BASIC_STATS":"true","COLUMN_STATS":{"db_id":"true","param_key":"true","param_value":"true"}}	0	0	0	0
+{"BASIC_STATS":"true","COLUMN_STATS":{"key":"true"}}	0	0	0	0
+{"BASIC_STATS":"true","COLUMN_STATS":{"param_key":"true","param_value":"true","serde_id":"true"}}	0	0	0	0
 PREHOOK: query: select COLUMN_STATS_ACCURATE, NUM_FILES, NUM_ROWS, RAW_DATA_SIZE, TOTAL_SIZE FROM PARTITION_STATS_VIEW where COLUMN_STATS_ACCURATE is not null order by NUM_FILES, NUM_ROWS, RAW_DATA_SIZE limit 5
 PREHOOK: type: QUERY
 PREHOOK: Input: sys@partition_params
@@ -3556,7 +3719,7 @@ STAGE PLANS:
             Execution mode: llap
             LLAP IO: no inputs
         Reducer 2 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
                 aggregations: max(VALUE._col0)
@@ -3592,27 +3755,35 @@ PREHOOK: Input: database:information_schema
 POSTHOOK: query: use INFORMATION_SCHEMA
 POSTHOOK: type: SWITCHDATABASE
 POSTHOOK: Input: database:information_schema
+Warning: Shuffle Join MERGEJOIN[20][tables = [d, t, p]] in Stage 'Reducer 2' is a cross product
 PREHOOK: query: select count(*) from SCHEMATA
 PREHOOK: type: QUERY
 PREHOOK: Input: information_schema@schemata
 PREHOOK: Input: sys@dbs
+PREHOOK: Input: sys@tbl_privs
+PREHOOK: Input: sys@tbls
 #### A masked pattern was here ####
 POSTHOOK: query: select count(*) from SCHEMATA
 POSTHOOK: type: QUERY
 POSTHOOK: Input: information_schema@schemata
 POSTHOOK: Input: sys@dbs
+POSTHOOK: Input: sys@tbl_privs
+POSTHOOK: Input: sys@tbls
 #### A masked pattern was here ####
 3
+Warning: Shuffle Join MERGEJOIN[38][tables = [t, d, p]] in Stage 'Reducer 3' is a cross product
 PREHOOK: query: select * from TABLES order by TABLE_SCHEMA, TABLE_NAME
 PREHOOK: type: QUERY
 PREHOOK: Input: information_schema@tables
 PREHOOK: Input: sys@dbs
+PREHOOK: Input: sys@tbl_privs
 PREHOOK: Input: sys@tbls
 #### A masked pattern was here ####
 POSTHOOK: query: select * from TABLES order by TABLE_SCHEMA, TABLE_NAME
 POSTHOOK: type: QUERY
 POSTHOOK: Input: information_schema@tables
 POSTHOOK: Input: sys@dbs
+POSTHOOK: Input: sys@tbl_privs
 POSTHOOK: Input: sys@tbls
 #### A masked pattern was here ####
 default	default	alltypesorc	BASE_TABLE	NULL	NULL	NULL	NULL	NULL	YES	NO	NULL
@@ -3672,6 +3843,7 @@ default	sys	wm_pools	BASE_TABLE	NULL	NULL	NULL	NULL	NULL	YES	NO	NULL
 default	sys	wm_pools_to_triggers	BASE_TABLE	NULL	NULL	NULL	NULL	NULL	YES	NO	NULL
 default	sys	wm_resourceplans	BASE_TABLE	NULL	NULL	NULL	NULL	NULL	YES	NO	NULL
 default	sys	wm_triggers	BASE_TABLE	NULL	NULL	NULL	NULL	NULL	YES	NO	NULL
+Warning: Shuffle Join MERGEJOIN[60][tables = [p, t, d, p2]] in Stage 'Reducer 4' is a cross product
 PREHOOK: query: select * from TABLE_PRIVILEGES order by GRANTOR, GRANTEE, TABLE_SCHEMA, TABLE_NAME, PRIVILEGE_TYPE limit 10
 PREHOOK: type: QUERY
 PREHOOK: Input: information_schema@table_privileges
@@ -3696,12 +3868,14 @@ hive_test_user	hive_test_user	default	default	moretypes	SELECT	YES	NO
 hive_test_user	hive_test_user	default	default	moretypes	UPDATE	YES	NO
 hive_test_user	hive_test_user	default	default	scr_txn	DELETE	YES	NO
 hive_test_user	hive_test_user	default	default	scr_txn	INSERT	YES	NO
+Warning: Shuffle Join MERGEJOIN[82][tables = [c, s, t, d, p]] in Stage 'Reducer 5' is a cross product
 PREHOOK: query: select * from COLUMNS where TABLE_NAME = 'alltypesorc' or TABLE_NAME = 'moretypes' order by TABLE_SCHEMA, TABLE_NAME, ORDINAL_POSITION
 PREHOOK: type: QUERY
 PREHOOK: Input: information_schema@columns
 PREHOOK: Input: sys@columns_v2
 PREHOOK: Input: sys@dbs
 PREHOOK: Input: sys@sds
+PREHOOK: Input: sys@tbl_col_privs
 PREHOOK: Input: sys@tbls
 #### A masked pattern was here ####
 POSTHOOK: query: select * from COLUMNS where TABLE_NAME = 'alltypesorc' or TABLE_NAME = 'moretypes' order by TABLE_SCHEMA, TABLE_NAME, ORDINAL_POSITION
@@ -3710,55 +3884,41 @@ POSTHOOK: Input: information_schema@columns
 POSTHOOK: Input: sys@columns_v2
 POSTHOOK: Input: sys@dbs
 POSTHOOK: Input: sys@sds
+POSTHOOK: Input: sys@tbl_col_privs
 POSTHOOK: Input: sys@tbls
 #### A masked pattern was here ####
-default	default	alltypesorc	ctinyint	0	NULL	YES	tinyint	NULL	NULL	3	10	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	12	NO	NO	NULL	NULL	NULL	NULL	NULL	NULL	NEVER	NULL	NO	NO	NULL	YES	tinyint	3	10
-default	default	alltypesorc	csmallint	1	NULL	YES	smallint	NULL	NULL	5	10	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	12	NO	NO	NULL	NULL	NULL	NULL	NULL	NULL	NEVER	NULL	NO	NO	NULL	YES	smallint	5	10
-default	default	alltypesorc	cint	2	NULL	YES	int	NULL	NULL	10	10	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	12	NO	NO	NULL	NULL	NULL	NULL	NULL	NULL	NEVER	NULL	NO	NO	NULL	YES	int	10	10
-default	default	alltypesorc	cbigint	3	NULL	YES	bigint	NULL	NULL	19	10	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	12	NO	NO	NULL	NULL	NULL	NULL	NULL	NULL	NEVER	NULL	NO	NO	NULL	YES	bigint	19	10
-default	default	alltypesorc	cfloat	4	NULL	YES	float	NULL	NULL	23	2	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	12	NO	NO	NULL	NULL	NULL	NULL	NULL	NULL	NEVER	NULL	NO	NO	NULL	YES	float	23	2
-default	default	alltypesorc	cdouble	5	NULL	YES	double	NULL	NULL	53	2	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	12	NO	NO	NULL	NULL	NULL	NULL	NULL	NULL	NEVER	NULL	NO	NO	NULL	YES	double	53	2
-default	default	alltypesorc	cstring1	6	NULL	YES	string	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	12	NO	NO	NULL	NULL	NULL	NULL	NULL	NULL	NEVER	NULL	NO	NO	NULL	YES	string	NULL	NULL
-default	default	alltypesorc	cstring2	7	NULL	YES	string	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	12	NO	NO	NULL	NULL	NULL	NULL	NULL	NULL	NEVER	NULL	NO	NO	NULL	YES	string	NULL	NULL
-default	default	alltypesorc	ctimestamp1	8	NULL	YES	timestamp	NULL	NULL	NULL	NULL	NULL	9	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	12	NO	NO	NULL	NULL	NULL	NULL	NULL	NULL	NEVER	NULL	NO	NO	NULL	YES	timestamp	NULL	NULL
-default	default	alltypesorc	ctimestamp2	9	NULL	YES	timestamp	NULL	NULL	NULL	NULL	NULL	9	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	12	NO	NO	NULL	NULL	NULL	NULL	NULL	NULL	NEVER	NULL	NO	NO	NULL	YES	timestamp	NULL	NULL
-default	default	alltypesorc	cboolean1	10	NULL	YES	boolean	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	12	NO	NO	NULL	NULL	NULL	NULL	NULL	NULL	NEVER	NULL	NO	NO	NULL	YES	boolean	NULL	NULL
-default	default	alltypesorc	cboolean2	11	NULL	YES	boolean	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	12	NO	NO	NULL	NULL	NULL	NULL	NULL	NULL	NEVER	NULL	NO	NO	NULL	YES	boolean	NULL	NULL
-default	default	moretypes	a	0	NULL	YES	decimal(10,2)	NULL	NULL	10	10	2	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	16	NO	NO	NULL	NULL	NULL	NULL	NULL	NULL	NEVER	NULL	NO	NO	NULL	YES	decimal(10,2)	10	10
-default	default	moretypes	b	1	NULL	YES	tinyint	NULL	NULL	3	10	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	16	NO	NO	NULL	NULL	NULL	NULL	NULL	NULL	NEVER	NULL	NO	NO	NULL	YES	tinyint	3	10
-default	default	moretypes	c	2	NULL	YES	smallint	NULL	NULL	5	10	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	16	NO	NO	NULL	NULL	NULL	NULL	NULL	NULL	NEVER	NULL	NO	NO	NULL	YES	smallint	5	10
-default	default	moretypes	d	3	NULL	YES	int	NULL	NULL	10	10	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	16	NO	NO	NULL	NULL	NULL	NULL	NULL	NULL	NEVER	NULL	NO	NO	NULL	YES	int	10	10
-default	default	moretypes	e	4	NULL	YES	bigint	NULL	NULL	19	10	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	16	NO	NO	NULL	NULL	NULL	NULL	NULL	NULL	NEVER	NULL	NO	NO	NULL	YES	bigint	19	10
-default	default	moretypes	f	5	NULL	YES	varchar(10)	10	10	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	16	NO	NO	NULL	NULL	NULL	NULL	NULL	NULL	NEVER	NULL	NO	NO	NULL	YES	varchar(10)	NULL	NULL
-default	default	moretypes	g	6	NULL	YES	char(3)	3	3	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	16	NO	NO	NULL	NULL	NULL	NULL	NULL	NULL	NEVER	NULL	NO	NO	NULL	YES	char(3)	NULL	NULL
+Warning: Shuffle Join MERGEJOIN[82][tables = [p, t, d, s, p2]] in Stage 'Reducer 5' is a cross product
 PREHOOK: query: select * from COLUMN_PRIVILEGES order by GRANTOR, GRANTEE, TABLE_SCHEMA, TABLE_NAME, COLUMN_NAME limit 10
 PREHOOK: type: QUERY
 PREHOOK: Input: information_schema@column_privileges
-PREHOOK: Input: sys@columns_v2
 PREHOOK: Input: sys@dbs
 PREHOOK: Input: sys@sds
 PREHOOK: Input: sys@tbl_col_privs
+PREHOOK: Input: sys@tbl_privs
 PREHOOK: Input: sys@tbls
 #### A masked pattern was here ####
 POSTHOOK: query: select * from COLUMN_PRIVILEGES order by GRANTOR, GRANTEE, TABLE_SCHEMA, TABLE_NAME, COLUMN_NAME limit 10
 POSTHOOK: type: QUERY
 POSTHOOK: Input: information_schema@column_privileges
-POSTHOOK: Input: sys@columns_v2
 POSTHOOK: Input: sys@dbs
 POSTHOOK: Input: sys@sds
 POSTHOOK: Input: sys@tbl_col_privs
+POSTHOOK: Input: sys@tbl_privs
 POSTHOOK: Input: sys@tbls
 #### A masked pattern was here ####
+Warning: Shuffle Join MERGEJOIN[38][tables = [d, t, p]] in Stage 'Reducer 3' is a cross product
 PREHOOK: query: select TABLE_SCHEMA, TABLE_NAME from views order by TABLE_SCHEMA, TABLE_NAME
 PREHOOK: type: QUERY
 PREHOOK: Input: information_schema@views
 PREHOOK: Input: sys@dbs
+PREHOOK: Input: sys@tbl_privs
 PREHOOK: Input: sys@tbls
 #### A masked pattern was here ####
 POSTHOOK: query: select TABLE_SCHEMA, TABLE_NAME from views order by TABLE_SCHEMA, TABLE_NAME
 POSTHOOK: type: QUERY
 POSTHOOK: Input: information_schema@views
 POSTHOOK: Input: sys@dbs
+POSTHOOK: Input: sys@tbl_privs
 POSTHOOK: Input: sys@tbls
 #### A masked pattern was here ####
 information_schema	column_privileges


[47/50] [abbrv] hive git commit: HIVE-19243 : Upgrade hadoop.version to 3.1.0 (Gour Saha via Sahil Takiar)

Posted by vg...@apache.org.
HIVE-19243 : Upgrade hadoop.version to 3.1.0 (Gour Saha via Sahil Takiar)

Signed-off-by: Ashutosh Chauhan <ha...@apache.org>


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

Branch: refs/heads/branch-3.0.0
Commit: 253f0ff28c948b6d42559b204c665db122dd151f
Parents: dc83c6f
Author: Gour Saha <gs...@hortonworks.com>
Authored: Thu Apr 19 07:18:21 2018 -0700
Committer: Vineet Garg <vg...@apache.org>
Committed: Tue May 15 15:05:17 2018 -0700

----------------------------------------------------------------------
 pom.xml | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/253f0ff2/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 9df8fb1..357a8d9 100644
--- a/pom.xml
+++ b/pom.xml
@@ -147,7 +147,7 @@
     <guava.version>19.0</guava.version>
     <groovy.version>2.4.11</groovy.version>
     <h2database.version>1.3.166</h2database.version>
-    <hadoop.version>3.0.0-beta1</hadoop.version>
+    <hadoop.version>3.1.0</hadoop.version>
     <hadoop.bin.path>${basedir}/${hive.path.to.root}/testutils/hadoop</hadoop.bin.path>
     <hamcrest.version>1.3</hamcrest.version>
     <hbase.version>2.0.0-alpha4</hbase.version>


[05/50] [abbrv] hive git commit: HIVE-19357: Vectorization: assert_true HiveException erroneously gets suppressed to NULL (Matt McCline, reviewed by Zoltan Haindrich)

Posted by vg...@apache.org.
HIVE-19357: Vectorization: assert_true HiveException erroneously gets suppressed to NULL (Matt McCline, reviewed by Zoltan Haindrich)


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

Branch: refs/heads/branch-3.0.0
Commit: 71be5aceeffcca8d7ba62d4f7066ebc11dacbc8a
Parents: 41f570a
Author: Matt McCline <mm...@hortonworks.com>
Authored: Tue May 8 01:04:39 2018 -0500
Committer: Matt McCline <mm...@hortonworks.com>
Committed: Tue May 8 01:05:48 2018 -0500

----------------------------------------------------------------------
 .../org/apache/hadoop/hive/conf/HiveConf.java   |   4 +
 .../vectorization/AbstractExpression.java       |   3 +-
 .../test/resources/testconfiguration.properties |   1 +
 .../ColumnArithmeticColumn.txt                  |   3 +-
 .../ColumnArithmeticColumnDecimal.txt           |   3 +-
 .../ColumnArithmeticScalar.txt                  |   3 +-
 .../ColumnArithmeticScalarDecimal.txt           |   3 +-
 .../ExpressionTemplates/ColumnCompareColumn.txt |   3 +-
 .../ExpressionTemplates/ColumnCompareScalar.txt |   3 +-
 .../ExpressionTemplates/ColumnDivideColumn.txt  |   3 +-
 .../ColumnDivideColumnDecimal.txt               |   3 +-
 .../ExpressionTemplates/ColumnDivideScalar.txt  |   3 +-
 .../ColumnDivideScalarDecimal.txt               |   3 +-
 .../ExpressionTemplates/ColumnUnaryFunc.txt     |   3 +-
 .../ExpressionTemplates/ColumnUnaryMinus.txt    |   3 +-
 ...eColumnArithmeticIntervalYearMonthColumn.txt |   3 +-
 ...eColumnArithmeticIntervalYearMonthScalar.txt |   3 +-
 .../DateColumnArithmeticTimestampColumn.txt     |   3 +-
 .../DateColumnArithmeticTimestampScalar.txt     |   3 +-
 ...eScalarArithmeticIntervalYearMonthColumn.txt |   3 +-
 .../DateScalarArithmeticTimestampColumn.txt     |   3 +-
 ...Decimal64ColumnArithmeticDecimal64Column.txt |   3 +-
 ...Decimal64ColumnArithmeticDecimal64Scalar.txt |   3 +-
 ...Decimal64ScalarArithmeticDecimal64Column.txt |   3 +-
 .../DecimalColumnUnaryFunc.txt                  |   3 +-
 .../ExpressionTemplates/FilterColumnBetween.txt |   3 +-
 .../FilterColumnBetweenDynamicValue.txt         |   2 +-
 .../FilterColumnCompareColumn.txt               |   3 +-
 .../FilterColumnCompareScalar.txt               |   3 +-
 .../FilterDecimalColumnBetween.txt              |   3 +-
 .../FilterDecimalColumnCompareDecimalColumn.txt |   3 +-
 .../FilterDecimalColumnCompareDecimalScalar.txt |   3 +-
 .../FilterDecimalScalarCompareDecimalColumn.txt |   3 +-
 ...erLongDoubleColumnCompareTimestampColumn.txt |   3 +-
 ...erLongDoubleScalarCompareTimestampColumn.txt |   3 +-
 .../FilterScalarCompareColumn.txt               |   3 +-
 .../FilterStringColumnBetween.txt               |   4 +-
 ...tringGroupColumnCompareStringGroupColumn.txt |   3 +-
 ...gGroupColumnCompareStringGroupScalarBase.txt |   3 +-
 ...gGroupScalarCompareStringGroupColumnBase.txt |   3 +-
 .../FilterTimestampColumnBetween.txt            |   3 +-
 ...erTimestampColumnCompareLongDoubleColumn.txt |   3 +-
 ...erTimestampColumnCompareLongDoubleScalar.txt |   3 +-
 ...terTimestampColumnCompareTimestampColumn.txt |   3 +-
 ...terTimestampColumnCompareTimestampScalar.txt |   3 +-
 ...terTimestampScalarCompareTimestampColumn.txt |   3 +-
 .../FilterTruncStringColumnBetween.txt          |   4 +-
 .../ExpressionTemplates/IfExprColumnScalar.txt  |   3 +-
 .../ExpressionTemplates/IfExprScalarColumn.txt  |   3 +-
 .../ExpressionTemplates/IfExprScalarScalar.txt  |   3 +-
 ...ervalYearMonthColumnArithmeticDateColumn.txt |   3 +-
 ...ervalYearMonthColumnArithmeticDateScalar.txt |   3 +-
 ...YearMonthColumnArithmeticTimestampColumn.txt |   3 +-
 ...YearMonthColumnArithmeticTimestampScalar.txt |   3 +-
 ...ervalYearMonthScalarArithmeticDateColumn.txt |   3 +-
 ...YearMonthScalarArithmeticTimestampColumn.txt |   3 +-
 .../LongDoubleColumnCompareTimestampColumn.txt  |   3 +-
 .../LongDoubleColumnCompareTimestampScalar.txt  |   3 +-
 .../LongDoubleScalarCompareTimestampColumn.txt  |   3 +-
 .../ScalarArithmeticColumn.txt                  |   3 +-
 .../ScalarArithmeticColumnDecimal.txt           |   3 +-
 .../ExpressionTemplates/ScalarCompareColumn.txt |   3 +-
 .../ExpressionTemplates/ScalarDivideColumn.txt  |   3 +-
 .../ScalarDivideColumnDecimal.txt               |   3 +-
 ...tringGroupColumnCompareStringGroupColumn.txt |   3 +-
 ...gGroupColumnCompareStringGroupScalarBase.txt |   3 +-
 ...gGroupScalarCompareStringGroupColumnBase.txt |   3 +-
 .../TimestampColumnArithmeticDateColumn.txt     |   3 +-
 .../TimestampColumnArithmeticDateScalar.txt     |   3 +-
 ...pColumnArithmeticIntervalYearMonthColumn.txt |   3 +-
 ...pColumnArithmeticIntervalYearMonthScalar.txt |   3 +-
 ...TimestampColumnArithmeticTimestampColumn.txt |   3 +-
 ...TimestampColumnArithmeticTimestampScalar.txt |   3 +-
 .../TimestampColumnCompareLongDoubleColumn.txt  |   3 +-
 .../TimestampColumnCompareLongDoubleScalar.txt  |   3 +-
 .../TimestampColumnCompareTimestampColumn.txt   |   3 +-
 .../TimestampColumnCompareTimestampScalar.txt   |   3 +-
 .../TimestampScalarArithmeticDateColumn.txt     |   3 +-
 ...pScalarArithmeticIntervalYearMonthColumn.txt |   3 +-
 ...TimestampScalarArithmeticTimestampColumn.txt |   3 +-
 .../TimestampScalarCompareTimestampColumn.txt   |   3 +-
 .../vectorization/TestTemplates/TestClass.txt   |   2 +-
 ...mnColumnFilterVectorExpressionEvaluation.txt |   2 +-
 ...erationVectorExpressionCheckedEvaluation.txt |   2 +-
 ...olumnOperationVectorExpressionEvaluation.txt |   2 +-
 ...mnScalarFilterVectorExpressionEvaluation.txt |   2 +-
 ...erationVectorExpressionCheckedEvaluation.txt |   2 +-
 ...calarOperationVectorExpressionEvaluation.txt |   2 +-
 .../ql/exec/vector/VectorizationContext.java    |   8 +-
 ...AbstractFilterStringColLikeStringScalar.java |   2 +-
 .../vector/expressions/CastDateToTimestamp.java |   3 +-
 .../expressions/CastDecimalToDecimal.java       |   3 +-
 .../expressions/CastDecimalToTimestamp.java     |   2 -
 .../expressions/CastDoubleToTimestamp.java      |   3 +-
 .../exec/vector/expressions/CastLongToDate.java |   3 +-
 .../vector/expressions/CastLongToTimestamp.java |   3 +-
 .../CastMillisecondsLongToTimestamp.java        |   3 +-
 .../vector/expressions/CastStringToDate.java    |   2 +-
 .../vector/expressions/CastStringToDecimal.java |   3 +-
 .../vector/expressions/CastStringToDouble.java  |   3 +-
 .../CastStringToIntervalDayTime.java            |   3 +-
 .../CastStringToIntervalYearMonth.java          |   3 +-
 .../vector/expressions/CastStringToLong.java    |   2 +-
 .../expressions/CastTimestampToBoolean.java     |   3 +-
 .../expressions/CastTimestampToDouble.java      |   3 +-
 .../vector/expressions/CastTimestampToLong.java |   3 +-
 .../ql/exec/vector/expressions/ColAndCol.java   |   3 +-
 .../ql/exec/vector/expressions/ColOrCol.java    |   3 +-
 .../expressions/DateColSubtractDateColumn.java  |   2 +-
 .../expressions/DateColSubtractDateScalar.java  |   2 +-
 .../DateScalarSubtractDateColumn.java           |   2 +-
 .../vector/expressions/DecimalColumnInList.java |   2 +-
 .../expressions/DecimalToStringUnaryUDF.java    |   3 +-
 .../vector/expressions/DoubleColumnInList.java  |   3 +-
 .../expressions/DoubleToStringUnaryUDF.java     |   3 +-
 .../vector/expressions/FilterColAndScalar.java  |   3 +-
 .../vector/expressions/FilterColOrScalar.java   |   3 +-
 .../expressions/FilterDecimalColumnInList.java  |   2 +-
 .../expressions/FilterDoubleColumnInList.java   |   2 +-
 .../vector/expressions/FilterExprAndExpr.java   |   3 +-
 .../vector/expressions/FilterExprOrExpr.java    |   3 +-
 .../expressions/FilterLongColumnInList.java     |   2 +-
 .../expressions/FilterScalarAndColumn.java      |   3 +-
 .../expressions/FilterScalarOrColumn.java       |   3 +-
 .../expressions/FilterStringColumnInList.java   |   3 +-
 .../expressions/FilterStructColumnInList.java   |   2 +-
 .../FilterTimestampColumnInList.java            |   2 +-
 .../vector/expressions/FuncDecimalToDouble.java |   3 +-
 .../vector/expressions/FuncDecimalToLong.java   |   3 +-
 .../expressions/FuncDecimalToTimestamp.java     |   3 +-
 .../vector/expressions/FuncDoubleToDecimal.java |   3 +-
 .../vector/expressions/FuncLongToDecimal.java   |   3 +-
 .../vector/expressions/FuncLongToString.java    |   2 +-
 .../ql/exec/vector/expressions/FuncRand.java    |   3 +-
 .../exec/vector/expressions/FuncRandNoSeed.java |   3 +-
 .../FuncRoundWithNumDigitsDecimalToDecimal.java |   3 +-
 .../vector/expressions/FuncStringToLong.java    |   3 +-
 .../expressions/FuncTimestampToDecimal.java     |   3 +-
 .../vector/expressions/FuncTimestampToLong.java |   3 +-
 .../vector/expressions/IdentityExpression.java  |   3 +-
 .../expressions/IfExprColumnCondExpr.java       |   3 +-
 .../vector/expressions/IfExprColumnNull.java    |   3 +-
 .../vector/expressions/IfExprCondExprBase.java  |   5 +-
 .../expressions/IfExprCondExprColumn.java       |   3 +-
 .../expressions/IfExprCondExprCondExpr.java     |   3 +-
 .../vector/expressions/IfExprCondExprNull.java  |   3 +-
 .../IfExprDoubleColumnDoubleColumn.java         |   3 +-
 .../IfExprIntervalDayTimeColumnColumn.java      |   3 +-
 .../IfExprIntervalDayTimeColumnScalar.java      |   3 +-
 .../IfExprIntervalDayTimeScalarColumn.java      |   3 +-
 .../IfExprIntervalDayTimeScalarScalar.java      |   3 +-
 .../expressions/IfExprLongColumnLongColumn.java |   3 +-
 .../vector/expressions/IfExprNullColumn.java    |   3 +-
 .../vector/expressions/IfExprNullCondExpr.java  |   3 +-
 .../exec/vector/expressions/IfExprNullNull.java |   3 +-
 ...fExprStringGroupColumnStringGroupColumn.java |   3 +-
 .../IfExprStringGroupColumnStringScalar.java    |   3 +-
 .../IfExprStringScalarStringGroupColumn.java    |   3 +-
 .../IfExprStringScalarStringScalar.java         |   3 +-
 .../IfExprTimestampColumnColumnBase.java        |   3 +-
 .../IfExprTimestampColumnScalarBase.java        |   3 +-
 .../IfExprTimestampScalarColumnBase.java        |   3 +-
 .../IfExprTimestampScalarScalarBase.java        |   3 +-
 .../ql/exec/vector/expressions/IsNotNull.java   |   3 +-
 .../hive/ql/exec/vector/expressions/IsNull.java |   3 +-
 .../vector/expressions/ListIndexColColumn.java  |   3 +-
 .../vector/expressions/ListIndexColScalar.java  |   3 +-
 .../expressions/LongColDivideLongColumn.java    |   3 +-
 .../expressions/LongColDivideLongScalar.java    |   3 +-
 .../expressions/LongColEqualLongColumn.java     |   3 +-
 .../expressions/LongColEqualLongScalar.java     |   3 +-
 .../LongColGreaterEqualLongColumn.java          |   3 +-
 .../LongColGreaterEqualLongScalar.java          |   3 +-
 .../expressions/LongColGreaterLongColumn.java   |   3 +-
 .../expressions/LongColGreaterLongScalar.java   |   3 +-
 .../expressions/LongColLessEqualLongColumn.java |   3 +-
 .../expressions/LongColLessEqualLongScalar.java |   3 +-
 .../expressions/LongColLessLongColumn.java      |   3 +-
 .../expressions/LongColLessLongScalar.java      |   3 +-
 .../expressions/LongColModuloLongColumn.java    |   5 +-
 .../LongColModuloLongColumnChecked.java         |   3 +-
 .../expressions/LongColNotEqualLongColumn.java  |   3 +-
 .../expressions/LongColNotEqualLongScalar.java  |   3 +-
 .../vector/expressions/LongColumnInList.java    |   3 +-
 .../expressions/LongScalarDivideLongColumn.java |   3 +-
 .../expressions/LongScalarEqualLongColumn.java  |   3 +-
 .../LongScalarGreaterEqualLongColumn.java       |   3 +-
 .../LongScalarGreaterLongColumn.java            |   3 +-
 .../LongScalarLessEqualLongColumn.java          |   3 +-
 .../expressions/LongScalarLessLongColumn.java   |   3 +-
 .../LongScalarNotEqualLongColumn.java           |   3 +-
 .../expressions/LongToStringUnaryUDF.java       |   3 +-
 .../expressions/MathFuncDoubleToDouble.java     |   3 +-
 .../expressions/MathFuncLongToDouble.java       |   3 +-
 .../vector/expressions/MathFuncLongToLong.java  |   3 +-
 .../hive/ql/exec/vector/expressions/NotCol.java |   3 +-
 .../ql/exec/vector/expressions/OctetLength.java |   3 +-
 .../vector/expressions/SelectColumnIsFalse.java |   3 +-
 .../expressions/SelectColumnIsNotNull.java      |   3 +-
 .../vector/expressions/SelectColumnIsNull.java  |   3 +-
 .../vector/expressions/SelectColumnIsTrue.java  |   3 +-
 .../SelectStringColLikeStringScalar.java        |   3 +-
 .../vector/expressions/StringColumnInList.java  |   2 +-
 .../StringGroupColConcatStringScalar.java       |   3 +-
 .../expressions/StringGroupConcatColCol.java    |   3 +-
 .../exec/vector/expressions/StringLength.java   |   3 +-
 .../StringScalarConcatStringGroupCol.java       |   3 +-
 .../expressions/StringSubstrColStart.java       |   3 +-
 .../expressions/StringSubstrColStartLen.java    |   3 +-
 .../exec/vector/expressions/StringUnaryUDF.java |   3 +-
 .../expressions/StringUnaryUDFDirect.java       |   3 +-
 .../vector/expressions/StructColumnInList.java  |   2 +-
 .../expressions/TimestampColumnInList.java      |   3 +-
 .../expressions/TimestampToStringUnaryUDF.java  |   3 +-
 .../exec/vector/expressions/VectorCoalesce.java |   3 +-
 .../ql/exec/vector/expressions/VectorElt.java   |   3 +-
 .../vector/expressions/VectorExpression.java    |   5 +-
 .../VectorInBloomFilterColDynamicValue.java     |   2 +-
 .../expressions/VectorUDFDateAddColCol.java     |   2 +-
 .../expressions/VectorUDFDateAddColScalar.java  |   2 +-
 .../expressions/VectorUDFDateAddScalarCol.java  |   2 +-
 .../expressions/VectorUDFDateDiffColCol.java    |   2 +-
 .../expressions/VectorUDFDateDiffColScalar.java |   3 +-
 .../expressions/VectorUDFDateDiffScalarCol.java |   3 +-
 .../expressions/VectorUDFMapIndexBaseCol.java   |   3 +-
 .../VectorUDFMapIndexBaseScalar.java            |   3 +-
 .../expressions/VectorUDFStructField.java       |   3 +-
 .../VectorUDFTimestampFieldDate.java            |   2 +-
 .../VectorUDFTimestampFieldString.java          |   2 +-
 .../VectorUDFTimestampFieldTimestamp.java       |   2 +-
 .../VectorMapJoinGenerateResultOperator.java    |   2 +-
 ...ectorMapJoinOuterGenerateResultOperator.java |   4 +-
 .../exec/vector/ptf/VectorPTFEvaluatorBase.java |   5 +-
 .../vector/ptf/VectorPTFEvaluatorCount.java     |   5 +-
 .../ptf/VectorPTFEvaluatorDecimalAvg.java       |   5 +-
 .../VectorPTFEvaluatorDecimalFirstValue.java    |   5 +-
 .../ptf/VectorPTFEvaluatorDecimalLastValue.java |   5 +-
 .../ptf/VectorPTFEvaluatorDecimalMax.java       |   5 +-
 .../ptf/VectorPTFEvaluatorDecimalMin.java       |   5 +-
 .../ptf/VectorPTFEvaluatorDecimalSum.java       |   5 +-
 .../vector/ptf/VectorPTFEvaluatorDenseRank.java |   5 +-
 .../vector/ptf/VectorPTFEvaluatorDoubleAvg.java |   5 +-
 .../ptf/VectorPTFEvaluatorDoubleFirstValue.java |   5 +-
 .../ptf/VectorPTFEvaluatorDoubleLastValue.java  |   5 +-
 .../vector/ptf/VectorPTFEvaluatorDoubleMax.java |   5 +-
 .../vector/ptf/VectorPTFEvaluatorDoubleMin.java |   5 +-
 .../vector/ptf/VectorPTFEvaluatorDoubleSum.java |   5 +-
 .../vector/ptf/VectorPTFEvaluatorLongAvg.java   |   5 +-
 .../ptf/VectorPTFEvaluatorLongFirstValue.java   |   5 +-
 .../ptf/VectorPTFEvaluatorLongLastValue.java    |   5 +-
 .../vector/ptf/VectorPTFEvaluatorLongMax.java   |   5 +-
 .../vector/ptf/VectorPTFEvaluatorLongMin.java   |   5 +-
 .../vector/ptf/VectorPTFEvaluatorLongSum.java   |   5 +-
 .../exec/vector/ptf/VectorPTFEvaluatorRank.java |   5 +-
 .../vector/ptf/VectorPTFEvaluatorRowNumber.java |   5 +-
 .../exec/vector/ptf/VectorPTFGroupBatches.java  |   7 +-
 .../ql/exec/vector/udf/VectorUDFAdaptor.java    |  29 ++--
 .../exec/vector/expressions/TestUnaryMinus.java |   7 +-
 .../TestVectorArithmeticExpressions.java        |  59 +++----
 .../TestVectorConditionalExpressions.java       |  26 +--
 .../expressions/TestVectorDateExpressions.java  |  18 +--
 .../TestVectorFilterExpressions.java            |  40 ++---
 .../TestVectorLogicalExpressions.java           |  39 ++---
 .../expressions/TestVectorMathFunctions.java    |  48 +++---
 .../TestVectorScalarColArithmetic.java          |  15 +-
 .../TestVectorStringExpressions.java            |  60 +++----
 .../vector/expressions/TestVectorTypeCasts.java |  34 ++--
 .../exec/vector/udf/TestVectorUDFAdaptor.java   |   6 +-
 .../clientpositive/vector_retry_failure.q       |  15 ++
 .../llap/vector_retry_failure.q.out             | 160 +++++++++++++++++++
 270 files changed, 906 insertions(+), 448 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
----------------------------------------------------------------------
diff --git a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
index 090c255..7b0fabe 100644
--- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
+++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
@@ -3567,6 +3567,10 @@ public class HiveConf extends Configuration {
         "This flag should be set to true to use overflow checked vector expressions when available.\n" +
         "For example, arithmetic expressions which can overflow the output data type can be evaluated using\n" +
         " checked vector expressions so that they produce same result as non-vectorized evaluation."),
+    HIVE_VECTORIZED_ADAPTOR_SUPPRESS_EVALUATE_EXCEPTIONS(
+		"hive.vectorized.adaptor.suppress.evaluate.exceptions", false,
+        "This flag should be set to true to suppress HiveException from the generic UDF function\n" +
+		"evaluate call and turn them into NULLs. Assume, by default, this is not needed"),
     HIVE_VECTORIZED_INPUT_FORMAT_SUPPORTS_ENABLED(
         "hive.vectorized.input.format.supports.enabled",
         "decimal_64",

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/itests/hive-jmh/src/main/java/org/apache/hive/benchmark/vectorization/AbstractExpression.java
----------------------------------------------------------------------
diff --git a/itests/hive-jmh/src/main/java/org/apache/hive/benchmark/vectorization/AbstractExpression.java b/itests/hive-jmh/src/main/java/org/apache/hive/benchmark/vectorization/AbstractExpression.java
index 681f173..b23d071 100644
--- a/itests/hive-jmh/src/main/java/org/apache/hive/benchmark/vectorization/AbstractExpression.java
+++ b/itests/hive-jmh/src/main/java/org/apache/hive/benchmark/vectorization/AbstractExpression.java
@@ -22,6 +22,7 @@ import org.apache.hadoop.hive.ql.exec.vector.DoubleColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.openjdk.jmh.annotations.Benchmark;
 import org.openjdk.jmh.annotations.BenchmarkMode;
 import org.openjdk.jmh.annotations.Fork;
@@ -58,7 +59,7 @@ public abstract class AbstractExpression {
   @Benchmark
   @Warmup(iterations = 2, time = 2, timeUnit = TimeUnit.MILLISECONDS)
   @Measurement(iterations = 2, time = 2, timeUnit = TimeUnit.MILLISECONDS)
-  public void bench() {
+  public void bench() throws HiveException {
     for (int i = 0; i < DEFAULT_ITER_TIME; i++) {
       rowBatch.selectedInUse = false;
       rowBatch.size = VectorizedRowBatch.DEFAULT_SIZE;

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/itests/src/test/resources/testconfiguration.properties
----------------------------------------------------------------------
diff --git a/itests/src/test/resources/testconfiguration.properties b/itests/src/test/resources/testconfiguration.properties
index 537bdba..aaa0fb1 100644
--- a/itests/src/test/resources/testconfiguration.properties
+++ b/itests/src/test/resources/testconfiguration.properties
@@ -762,6 +762,7 @@ minillaplocal.query.files=\
   vector_partitioned_date_time.q,\
   vector_ptf_1.q,\
   vector_ptf_part_simple.q,\
+  vector_retry_failure.q,\
   vector_reuse_scratchcols.q,\
   vector_string_decimal.q,\
   vector_udf_adaptor_1.q,\

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/gen/vectorization/ExpressionTemplates/ColumnArithmeticColumn.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/ColumnArithmeticColumn.txt b/ql/src/gen/vectorization/ExpressionTemplates/ColumnArithmeticColumn.txt
index cb29953..f4a95ac 100644
--- a/ql/src/gen/vectorization/ExpressionTemplates/ColumnArithmeticColumn.txt
+++ b/ql/src/gen/vectorization/ExpressionTemplates/ColumnArithmeticColumn.txt
@@ -24,6 +24,7 @@ import org.apache.hadoop.hive.ql.exec.vector.expressions.NullUtil;
 import org.apache.hadoop.hive.ql.exec.vector.*;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 /**
  * Generated from template ColumnArithmeticColumn.txt, which covers binary arithmetic 
@@ -51,7 +52,7 @@ public class <ClassName> extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     // return immediately if batch is empty
     final int n = batch.size;

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/gen/vectorization/ExpressionTemplates/ColumnArithmeticColumnDecimal.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/ColumnArithmeticColumnDecimal.txt b/ql/src/gen/vectorization/ExpressionTemplates/ColumnArithmeticColumnDecimal.txt
index d7c7fb4..23ede32 100644
--- a/ql/src/gen/vectorization/ExpressionTemplates/ColumnArithmeticColumnDecimal.txt
+++ b/ql/src/gen/vectorization/ExpressionTemplates/ColumnArithmeticColumnDecimal.txt
@@ -25,6 +25,7 @@ import org.apache.hadoop.hive.ql.exec.vector.expressions.NullUtil;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.DecimalUtil;
 import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
 import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 /**
  * Generated from template ColumnArithmeticColumnDecimal.txt, which covers binary arithmetic 
@@ -52,7 +53,7 @@ public class <ClassName> extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     // return immediately if batch is empty
     final int n = batch.size;

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/gen/vectorization/ExpressionTemplates/ColumnArithmeticScalar.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/ColumnArithmeticScalar.txt b/ql/src/gen/vectorization/ExpressionTemplates/ColumnArithmeticScalar.txt
index 00b8d96..cd1e46a 100644
--- a/ql/src/gen/vectorization/ExpressionTemplates/ColumnArithmeticScalar.txt
+++ b/ql/src/gen/vectorization/ExpressionTemplates/ColumnArithmeticScalar.txt
@@ -27,6 +27,7 @@ import org.apache.hadoop.hive.ql.exec.vector.<OutputColumnVectorType>;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.NullUtil;
 import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 /**
  * Generated from template ColumnArithmeticScalar.txt, which covers binary arithmetic 
@@ -54,7 +55,7 @@ public class <ClassName> extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     // return immediately if batch is empty
     final int n = batch.size;

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/gen/vectorization/ExpressionTemplates/ColumnArithmeticScalarDecimal.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/ColumnArithmeticScalarDecimal.txt b/ql/src/gen/vectorization/ExpressionTemplates/ColumnArithmeticScalarDecimal.txt
index 9d0365e..1824edb 100644
--- a/ql/src/gen/vectorization/ExpressionTemplates/ColumnArithmeticScalarDecimal.txt
+++ b/ql/src/gen/vectorization/ExpressionTemplates/ColumnArithmeticScalarDecimal.txt
@@ -27,6 +27,7 @@ import org.apache.hadoop.hive.ql.exec.vector.expressions.DecimalUtil;
 import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
 import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
 import org.apache.hadoop.hive.common.type.HiveDecimal;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 /**
  * Generated from template ColumnArithmeticScalarDecimal.txt, which covers binary arithmetic 
@@ -54,7 +55,7 @@ public class <ClassName> extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     // return immediately if batch is empty
     final int n = batch.size;

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/gen/vectorization/ExpressionTemplates/ColumnCompareColumn.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/ColumnCompareColumn.txt b/ql/src/gen/vectorization/ExpressionTemplates/ColumnCompareColumn.txt
index 14a0993..56d96b8 100644
--- a/ql/src/gen/vectorization/ExpressionTemplates/ColumnCompareColumn.txt
+++ b/ql/src/gen/vectorization/ExpressionTemplates/ColumnCompareColumn.txt
@@ -23,6 +23,7 @@ import org.apache.hadoop.hive.ql.exec.vector.expressions.NullUtil;
 import org.apache.hadoop.hive.ql.exec.vector.*;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 /**
  * Generated from template ColumnArithmeticColumn.txt, which covers binary arithmetic 
@@ -50,7 +51,7 @@ public class <ClassName> extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     // return immediately if batch is empty
     final int n = batch.size;

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/gen/vectorization/ExpressionTemplates/ColumnCompareScalar.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/ColumnCompareScalar.txt b/ql/src/gen/vectorization/ExpressionTemplates/ColumnCompareScalar.txt
index 79b1bbd..9556066 100644
--- a/ql/src/gen/vectorization/ExpressionTemplates/ColumnCompareScalar.txt
+++ b/ql/src/gen/vectorization/ExpressionTemplates/ColumnCompareScalar.txt
@@ -25,6 +25,7 @@ import org.apache.hadoop.hive.ql.exec.vector.<InputColumnVectorType>;
 import org.apache.hadoop.hive.ql.exec.vector.<OutputColumnVectorType>;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 /**
  * Generated from template ColumnCompareScalar.txt, which covers binary comparison 
@@ -53,7 +54,7 @@ public class <ClassName> extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     // return immediately if batch is empty
     final int n = batch.size;

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/gen/vectorization/ExpressionTemplates/ColumnDivideColumn.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/ColumnDivideColumn.txt b/ql/src/gen/vectorization/ExpressionTemplates/ColumnDivideColumn.txt
index 1e20dba..c4a76ae 100644
--- a/ql/src/gen/vectorization/ExpressionTemplates/ColumnDivideColumn.txt
+++ b/ql/src/gen/vectorization/ExpressionTemplates/ColumnDivideColumn.txt
@@ -24,6 +24,7 @@ import org.apache.hadoop.hive.ql.exec.vector.expressions.NullUtil;
 import org.apache.hadoop.hive.ql.exec.vector.*;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 /**
  * Generated from template ColumnDivideColumn.txt, which covers division and modulo
@@ -51,7 +52,7 @@ public class <ClassName> extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     // return immediately if batch is empty
     final int n = batch.size;

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/gen/vectorization/ExpressionTemplates/ColumnDivideColumnDecimal.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/ColumnDivideColumnDecimal.txt b/ql/src/gen/vectorization/ExpressionTemplates/ColumnDivideColumnDecimal.txt
index c8a2665..8faf095 100644
--- a/ql/src/gen/vectorization/ExpressionTemplates/ColumnDivideColumnDecimal.txt
+++ b/ql/src/gen/vectorization/ExpressionTemplates/ColumnDivideColumnDecimal.txt
@@ -25,6 +25,7 @@ import org.apache.hadoop.hive.ql.exec.vector.expressions.NullUtil;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.DecimalUtil;
 import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
 import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 /**
  * Generated from template ColumnArithmeticColumnDecimal.txt, which covers binary arithmetic
@@ -52,7 +53,7 @@ public class <ClassName> extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     // return immediately if batch is empty
     final int n = batch.size;

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/gen/vectorization/ExpressionTemplates/ColumnDivideScalar.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/ColumnDivideScalar.txt b/ql/src/gen/vectorization/ExpressionTemplates/ColumnDivideScalar.txt
index 74fd0e2..3b0e147 100644
--- a/ql/src/gen/vectorization/ExpressionTemplates/ColumnDivideScalar.txt
+++ b/ql/src/gen/vectorization/ExpressionTemplates/ColumnDivideScalar.txt
@@ -26,6 +26,7 @@ import org.apache.hadoop.hive.ql.exec.vector.<OutputColumnVectorType>;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.NullUtil;
 import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 /**
  * Generated from template ColumnArithmeticScalar.txt, which covers binary arithmetic
@@ -53,7 +54,7 @@ public class <ClassName> extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     // return immediately if batch is empty
     final int n = batch.size;

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/gen/vectorization/ExpressionTemplates/ColumnDivideScalarDecimal.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/ColumnDivideScalarDecimal.txt b/ql/src/gen/vectorization/ExpressionTemplates/ColumnDivideScalarDecimal.txt
index 15d21d0..0bd7c00 100644
--- a/ql/src/gen/vectorization/ExpressionTemplates/ColumnDivideScalarDecimal.txt
+++ b/ql/src/gen/vectorization/ExpressionTemplates/ColumnDivideScalarDecimal.txt
@@ -27,6 +27,7 @@ import org.apache.hadoop.hive.ql.exec.vector.expressions.DecimalUtil;
 import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
 import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
 import org.apache.hadoop.hive.common.type.HiveDecimal;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 /**
  * Generated from template ColumnDivideScalarDecimal.txt, which covers binary arithmetic
@@ -54,7 +55,7 @@ public class <ClassName> extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     // return immediately if batch is empty
     final int n = batch.size;

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/gen/vectorization/ExpressionTemplates/ColumnUnaryFunc.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/ColumnUnaryFunc.txt b/ql/src/gen/vectorization/ExpressionTemplates/ColumnUnaryFunc.txt
index cead258..3ac896c 100644
--- a/ql/src/gen/vectorization/ExpressionTemplates/ColumnUnaryFunc.txt
+++ b/ql/src/gen/vectorization/ExpressionTemplates/ColumnUnaryFunc.txt
@@ -25,6 +25,7 @@ import org.apache.hadoop.hive.ql.exec.vector.expressions.MathExpr;
 import org.apache.hadoop.hive.ql.exec.vector.*;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 public class <ClassName> extends VectorExpression {
   private static final long serialVersionUID = 1L;
@@ -44,7 +45,7 @@ public class <ClassName> extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     // return immediately if batch is empty
     final int n = batch.size;

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/gen/vectorization/ExpressionTemplates/ColumnUnaryMinus.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/ColumnUnaryMinus.txt b/ql/src/gen/vectorization/ExpressionTemplates/ColumnUnaryMinus.txt
index 53e4bb4..8fa8e44 100644
--- a/ql/src/gen/vectorization/ExpressionTemplates/ColumnUnaryMinus.txt
+++ b/ql/src/gen/vectorization/ExpressionTemplates/ColumnUnaryMinus.txt
@@ -25,6 +25,7 @@ import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
 import org.apache.hadoop.hive.ql.exec.vector.*;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 /**
  * Generated from template ColumnUnaryMinus.txt, which covers unary negation operator. 
@@ -48,7 +49,7 @@ public class <ClassName> extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     // return immediately if batch is empty
     final int n = batch.size;

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/gen/vectorization/ExpressionTemplates/DateColumnArithmeticIntervalYearMonthColumn.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/DateColumnArithmeticIntervalYearMonthColumn.txt b/ql/src/gen/vectorization/ExpressionTemplates/DateColumnArithmeticIntervalYearMonthColumn.txt
index bf1128a..32dd6ed 100644
--- a/ql/src/gen/vectorization/ExpressionTemplates/DateColumnArithmeticIntervalYearMonthColumn.txt
+++ b/ql/src/gen/vectorization/ExpressionTemplates/DateColumnArithmeticIntervalYearMonthColumn.txt
@@ -27,6 +27,7 @@ import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
 import org.apache.hadoop.hive.ql.util.DateTimeMath;
 import org.apache.hadoop.hive.serde2.io.DateWritable;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 /**
  * Generated from template DateColumnArithmeticIntervalYearMonthColumn.txt, which covers binary arithmetic
@@ -59,7 +60,7 @@ public class <ClassName> extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     // return immediately if batch is empty
     final int n = batch.size;

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/gen/vectorization/ExpressionTemplates/DateColumnArithmeticIntervalYearMonthScalar.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/DateColumnArithmeticIntervalYearMonthScalar.txt b/ql/src/gen/vectorization/ExpressionTemplates/DateColumnArithmeticIntervalYearMonthScalar.txt
index 172bd39..94c0c5c 100644
--- a/ql/src/gen/vectorization/ExpressionTemplates/DateColumnArithmeticIntervalYearMonthScalar.txt
+++ b/ql/src/gen/vectorization/ExpressionTemplates/DateColumnArithmeticIntervalYearMonthScalar.txt
@@ -30,6 +30,7 @@ import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
 import org.apache.hadoop.hive.ql.exec.vector.*;
 import org.apache.hadoop.hive.ql.util.DateTimeMath;
 import org.apache.hadoop.hive.serde2.io.DateWritable;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 /**
  * Generated from template DateColumnArithmeticIntervalYearMonthScalar.txt, which covers binary arithmetic
@@ -61,7 +62,7 @@ public class <ClassName> extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     // return immediately if batch is empty
     final int n = batch.size;

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/gen/vectorization/ExpressionTemplates/DateColumnArithmeticTimestampColumn.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/DateColumnArithmeticTimestampColumn.txt b/ql/src/gen/vectorization/ExpressionTemplates/DateColumnArithmeticTimestampColumn.txt
index a515319..96c525d 100644
--- a/ql/src/gen/vectorization/ExpressionTemplates/DateColumnArithmeticTimestampColumn.txt
+++ b/ql/src/gen/vectorization/ExpressionTemplates/DateColumnArithmeticTimestampColumn.txt
@@ -29,6 +29,7 @@ import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
 import org.apache.hadoop.hive.ql.util.DateTimeMath;
 import org.apache.hadoop.hive.serde2.io.DateWritable;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 /**
  * Generated from template DateColumnArithmeticTimestampColumn.txt, a class
@@ -59,7 +60,7 @@ public class <ClassName> extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     // return immediately if batch is empty
     final int n = batch.size;

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/gen/vectorization/ExpressionTemplates/DateColumnArithmeticTimestampScalar.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/DateColumnArithmeticTimestampScalar.txt b/ql/src/gen/vectorization/ExpressionTemplates/DateColumnArithmeticTimestampScalar.txt
index 0bae7da..fb22992 100644
--- a/ql/src/gen/vectorization/ExpressionTemplates/DateColumnArithmeticTimestampScalar.txt
+++ b/ql/src/gen/vectorization/ExpressionTemplates/DateColumnArithmeticTimestampScalar.txt
@@ -30,6 +30,7 @@ import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
 import org.apache.hadoop.hive.ql.exec.vector.*;
 import org.apache.hadoop.hive.ql.util.DateTimeMath;
 import org.apache.hadoop.hive.serde2.io.DateWritable;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 /**
  * Generated from template DateColumnArithmeticTimestampScalarBase.txt, a base class
@@ -60,7 +61,7 @@ public class <ClassName> extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     // return immediately if batch is empty
     final int n = batch.size;

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/gen/vectorization/ExpressionTemplates/DateScalarArithmeticIntervalYearMonthColumn.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/DateScalarArithmeticIntervalYearMonthColumn.txt b/ql/src/gen/vectorization/ExpressionTemplates/DateScalarArithmeticIntervalYearMonthColumn.txt
index 42046e0..0c8ec9c 100644
--- a/ql/src/gen/vectorization/ExpressionTemplates/DateScalarArithmeticIntervalYearMonthColumn.txt
+++ b/ql/src/gen/vectorization/ExpressionTemplates/DateScalarArithmeticIntervalYearMonthColumn.txt
@@ -38,6 +38,7 @@ import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.NullUtil;
 import org.apache.hadoop.hive.ql.util.DateTimeMath;
 import org.apache.hadoop.hive.serde2.io.DateWritable;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 /**
  * Generated from template DateTimeScalarArithmeticIntervalYearMonthColumn.txt.
@@ -75,7 +76,7 @@ public class <ClassName> extends VectorExpression {
    *
    * @batch a package of rows with each column stored in a vector
    */
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     // return immediately if batch is empty
     final int n = batch.size;

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/gen/vectorization/ExpressionTemplates/DateScalarArithmeticTimestampColumn.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/DateScalarArithmeticTimestampColumn.txt b/ql/src/gen/vectorization/ExpressionTemplates/DateScalarArithmeticTimestampColumn.txt
index 191eaa6..ef8f2a3 100644
--- a/ql/src/gen/vectorization/ExpressionTemplates/DateScalarArithmeticTimestampColumn.txt
+++ b/ql/src/gen/vectorization/ExpressionTemplates/DateScalarArithmeticTimestampColumn.txt
@@ -37,6 +37,7 @@ import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.NullUtil;
 import org.apache.hadoop.hive.ql.util.DateTimeMath;
 import org.apache.hadoop.hive.serde2.io.DateWritable;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 /**
  * Generated from template DateTimeScalarArithmeticTimestampColumnBase.txt.
@@ -74,7 +75,7 @@ public class <ClassName> extends VectorExpression {
    *
    * @batch a package of rows with each column stored in a vector
    */
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     // return immediately if batch is empty
     final int n = batch.size;

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/gen/vectorization/ExpressionTemplates/Decimal64ColumnArithmeticDecimal64Column.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/Decimal64ColumnArithmeticDecimal64Column.txt b/ql/src/gen/vectorization/ExpressionTemplates/Decimal64ColumnArithmeticDecimal64Column.txt
index 5b643bb..25ebae4 100644
--- a/ql/src/gen/vectorization/ExpressionTemplates/Decimal64ColumnArithmeticDecimal64Column.txt
+++ b/ql/src/gen/vectorization/ExpressionTemplates/Decimal64ColumnArithmeticDecimal64Column.txt
@@ -25,6 +25,7 @@ import org.apache.hadoop.hive.ql.exec.vector.expressions.NullUtil;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.Decimal64Util;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
 import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 /**
  * Generated from template Decimal64ColumnArithmeticDecimal64Column.txt, which covers
@@ -52,7 +53,7 @@ public class <ClassName> extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     // return immediately if batch is empty
     final int n = batch.size;

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/gen/vectorization/ExpressionTemplates/Decimal64ColumnArithmeticDecimal64Scalar.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/Decimal64ColumnArithmeticDecimal64Scalar.txt b/ql/src/gen/vectorization/ExpressionTemplates/Decimal64ColumnArithmeticDecimal64Scalar.txt
index fa091d4..35d9f06 100644
--- a/ql/src/gen/vectorization/ExpressionTemplates/Decimal64ColumnArithmeticDecimal64Scalar.txt
+++ b/ql/src/gen/vectorization/ExpressionTemplates/Decimal64ColumnArithmeticDecimal64Scalar.txt
@@ -27,6 +27,7 @@ import org.apache.hadoop.hive.ql.exec.vector.expressions.Decimal64Util;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
 import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
 import org.apache.hadoop.hive.serde2.typeinfo.DecimalTypeInfo;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 /**
  * Generated from template ColumnArithmeticScalar.txt, which covers decimal64 arithmetic
@@ -54,7 +55,7 @@ public class <ClassName> extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     // return immediately if batch is empty
     final int n = batch.size;

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/gen/vectorization/ExpressionTemplates/Decimal64ScalarArithmeticDecimal64Column.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/Decimal64ScalarArithmeticDecimal64Column.txt b/ql/src/gen/vectorization/ExpressionTemplates/Decimal64ScalarArithmeticDecimal64Column.txt
index db813b7..ca5b4bc 100644
--- a/ql/src/gen/vectorization/ExpressionTemplates/Decimal64ScalarArithmeticDecimal64Column.txt
+++ b/ql/src/gen/vectorization/ExpressionTemplates/Decimal64ScalarArithmeticDecimal64Column.txt
@@ -27,6 +27,7 @@ import org.apache.hadoop.hive.ql.exec.vector.expressions.Decimal64Util;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
 import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
 import org.apache.hadoop.hive.serde2.typeinfo.DecimalTypeInfo;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 /**
  * Generated from template Decimal64ScalarArithmeticDecimal64Column.txt.
@@ -55,7 +56,7 @@ public class <ClassName> extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     // return immediately if batch is empty
     final int n = batch.size;

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/gen/vectorization/ExpressionTemplates/DecimalColumnUnaryFunc.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/DecimalColumnUnaryFunc.txt b/ql/src/gen/vectorization/ExpressionTemplates/DecimalColumnUnaryFunc.txt
index 5e684b4..aaec7e0 100644
--- a/ql/src/gen/vectorization/ExpressionTemplates/DecimalColumnUnaryFunc.txt
+++ b/ql/src/gen/vectorization/ExpressionTemplates/DecimalColumnUnaryFunc.txt
@@ -29,6 +29,7 @@ import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
 import org.apache.hadoop.hive.ql.exec.vector.DecimalColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.DecimalUtil;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 public class <ClassName> extends VectorExpression {
   private static final long serialVersionUID = 1L;
@@ -48,7 +49,7 @@ public class <ClassName> extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     // return immediately if batch is empty
     final int n = batch.size;

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/gen/vectorization/ExpressionTemplates/FilterColumnBetween.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/FilterColumnBetween.txt b/ql/src/gen/vectorization/ExpressionTemplates/FilterColumnBetween.txt
index 55eb50e..0664cbf 100644
--- a/ql/src/gen/vectorization/ExpressionTemplates/FilterColumnBetween.txt
+++ b/ql/src/gen/vectorization/ExpressionTemplates/FilterColumnBetween.txt
@@ -22,6 +22,7 @@ import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
 import org.apache.hadoop.hive.ql.exec.vector.<InputColumnVectorType>;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 /**
  * Generated from template FilterColumnBetween.txt, which covers [NOT] BETWEEN filter 
@@ -55,7 +56,7 @@ public class <ClassName> extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     // return immediately if batch is empty
     final int n = batch.size;

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/gen/vectorization/ExpressionTemplates/FilterColumnBetweenDynamicValue.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/FilterColumnBetweenDynamicValue.txt b/ql/src/gen/vectorization/ExpressionTemplates/FilterColumnBetweenDynamicValue.txt
index b5a36df..5242bbd 100644
--- a/ql/src/gen/vectorization/ExpressionTemplates/FilterColumnBetweenDynamicValue.txt
+++ b/ql/src/gen/vectorization/ExpressionTemplates/FilterColumnBetweenDynamicValue.txt
@@ -83,7 +83,7 @@ public class <ClassName> extends Filter<TypeName>ColumnBetween {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
     if (!initialized) {
       Object lVal = leftDynamicValue.getValue();
       Object rVal = rightDynamicValue.getValue();

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/gen/vectorization/ExpressionTemplates/FilterColumnCompareColumn.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/FilterColumnCompareColumn.txt b/ql/src/gen/vectorization/ExpressionTemplates/FilterColumnCompareColumn.txt
index e458992..be9bbb2 100644
--- a/ql/src/gen/vectorization/ExpressionTemplates/FilterColumnCompareColumn.txt
+++ b/ql/src/gen/vectorization/ExpressionTemplates/FilterColumnCompareColumn.txt
@@ -23,6 +23,7 @@ import org.apache.hadoop.hive.ql.exec.vector.expressions.NullUtil;
 import org.apache.hadoop.hive.ql.exec.vector.*;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 /**
  * Generated from template FilterColumnCompareColumn.txt, which covers binary comparison 
@@ -51,7 +52,7 @@ public class <ClassName> extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     // return immediately if batch is empty
     int n = batch.size;

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/gen/vectorization/ExpressionTemplates/FilterColumnCompareScalar.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/FilterColumnCompareScalar.txt b/ql/src/gen/vectorization/ExpressionTemplates/FilterColumnCompareScalar.txt
index c955c06..8b6f978 100644
--- a/ql/src/gen/vectorization/ExpressionTemplates/FilterColumnCompareScalar.txt
+++ b/ql/src/gen/vectorization/ExpressionTemplates/FilterColumnCompareScalar.txt
@@ -22,6 +22,7 @@ import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
 import org.apache.hadoop.hive.ql.exec.vector.<InputColumnVectorType>;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 /**
  * Generated from template FilterColumnCompareScalar.txt, which covers binary comparison 
@@ -49,7 +50,7 @@ public class <ClassName> extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     // return immediately if batch is empty
     final int n = batch.size;

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/gen/vectorization/ExpressionTemplates/FilterDecimalColumnBetween.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/FilterDecimalColumnBetween.txt b/ql/src/gen/vectorization/ExpressionTemplates/FilterDecimalColumnBetween.txt
index f42668c..c7cfc4d 100644
--- a/ql/src/gen/vectorization/ExpressionTemplates/FilterDecimalColumnBetween.txt
+++ b/ql/src/gen/vectorization/ExpressionTemplates/FilterDecimalColumnBetween.txt
@@ -25,6 +25,7 @@ import org.apache.hadoop.hive.ql.exec.vector.expressions.DecimalUtil;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
 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;
 
 /**
  * Generated from template FilterDecimalColumnBetween.txt, which covers [NOT] BETWEEN filter
@@ -58,7 +59,7 @@ public class <ClassName> extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     // return immediately if batch is empty
     final int n = batch.size;

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/gen/vectorization/ExpressionTemplates/FilterDecimalColumnCompareDecimalColumn.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/FilterDecimalColumnCompareDecimalColumn.txt b/ql/src/gen/vectorization/ExpressionTemplates/FilterDecimalColumnCompareDecimalColumn.txt
index 77fe7ae..ae2bb17 100644
--- a/ql/src/gen/vectorization/ExpressionTemplates/FilterDecimalColumnCompareDecimalColumn.txt
+++ b/ql/src/gen/vectorization/ExpressionTemplates/FilterDecimalColumnCompareDecimalColumn.txt
@@ -23,6 +23,7 @@ import org.apache.hadoop.hive.ql.exec.vector.DecimalColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
 import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 /**
  * Generated from template FilterDecimalColumnCompareColumn.txt, which covers binary comparison
@@ -51,7 +52,7 @@ public class <ClassName> extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     // return immediately if batch is empty
     final int n = batch.size;

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/gen/vectorization/ExpressionTemplates/FilterDecimalColumnCompareDecimalScalar.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/FilterDecimalColumnCompareDecimalScalar.txt b/ql/src/gen/vectorization/ExpressionTemplates/FilterDecimalColumnCompareDecimalScalar.txt
index 078b132..5e59c03 100644
--- a/ql/src/gen/vectorization/ExpressionTemplates/FilterDecimalColumnCompareDecimalScalar.txt
+++ b/ql/src/gen/vectorization/ExpressionTemplates/FilterDecimalColumnCompareDecimalScalar.txt
@@ -25,6 +25,7 @@ import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
 import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
 import org.apache.hadoop.hive.common.type.HiveDecimal;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 /**
  * This is a generated class to evaluate a <OperatorSymbol> comparison on a vector of decimal
@@ -52,7 +53,7 @@ public class <ClassName> extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     // return immediately if batch is empty
     final int n = batch.size;

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/gen/vectorization/ExpressionTemplates/FilterDecimalScalarCompareDecimalColumn.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/FilterDecimalScalarCompareDecimalColumn.txt b/ql/src/gen/vectorization/ExpressionTemplates/FilterDecimalScalarCompareDecimalColumn.txt
index 20dbaba..00d0042 100644
--- a/ql/src/gen/vectorization/ExpressionTemplates/FilterDecimalScalarCompareDecimalColumn.txt
+++ b/ql/src/gen/vectorization/ExpressionTemplates/FilterDecimalScalarCompareDecimalColumn.txt
@@ -25,6 +25,7 @@ import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
 import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
 import org.apache.hadoop.hive.common.type.HiveDecimal;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 /**
  * This is a generated class to evaluate a <OperatorSymbol> comparison on a vector of decimal
@@ -52,7 +53,7 @@ public class <ClassName> extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     // return immediately if batch is empty
     final int n = batch.size;

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/gen/vectorization/ExpressionTemplates/FilterLongDoubleColumnCompareTimestampColumn.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/FilterLongDoubleColumnCompareTimestampColumn.txt b/ql/src/gen/vectorization/ExpressionTemplates/FilterLongDoubleColumnCompareTimestampColumn.txt
index 4afed54..4e78fd6 100644
--- a/ql/src/gen/vectorization/ExpressionTemplates/FilterLongDoubleColumnCompareTimestampColumn.txt
+++ b/ql/src/gen/vectorization/ExpressionTemplates/FilterLongDoubleColumnCompareTimestampColumn.txt
@@ -26,6 +26,7 @@ import org.apache.hadoop.hive.ql.exec.vector.expressions.NullUtil;
 import org.apache.hadoop.hive.ql.exec.vector.*;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 /**
  * Generated from template FilterColumnCompareColumn.txt, which covers binary comparison
@@ -54,7 +55,7 @@ public class <ClassName> extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     // return immediately if batch is empty
     int n = batch.size;

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/gen/vectorization/ExpressionTemplates/FilterLongDoubleScalarCompareTimestampColumn.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/FilterLongDoubleScalarCompareTimestampColumn.txt b/ql/src/gen/vectorization/ExpressionTemplates/FilterLongDoubleScalarCompareTimestampColumn.txt
index 8f8104d..69cf579 100644
--- a/ql/src/gen/vectorization/ExpressionTemplates/FilterLongDoubleScalarCompareTimestampColumn.txt
+++ b/ql/src/gen/vectorization/ExpressionTemplates/FilterLongDoubleScalarCompareTimestampColumn.txt
@@ -26,6 +26,7 @@ import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
 import org.apache.hadoop.hive.ql.exec.vector.TimestampColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 /**
  * Generated from template FilterScalarCompareTimestampColumn.txt, which covers comparison
@@ -55,7 +56,7 @@ public class <ClassName> extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     // return immediately if batch is empty
     final int n = batch.size;

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/gen/vectorization/ExpressionTemplates/FilterScalarCompareColumn.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/FilterScalarCompareColumn.txt b/ql/src/gen/vectorization/ExpressionTemplates/FilterScalarCompareColumn.txt
index 28b5704..2ff9e98 100644
--- a/ql/src/gen/vectorization/ExpressionTemplates/FilterScalarCompareColumn.txt
+++ b/ql/src/gen/vectorization/ExpressionTemplates/FilterScalarCompareColumn.txt
@@ -22,6 +22,7 @@ import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
 import org.apache.hadoop.hive.ql.exec.vector.<InputColumnVectorType>;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 /**
  * Generated from template FilterScalarCompareColumn.txt, which covers binary comparison 
@@ -50,7 +51,7 @@ public class <ClassName> extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     // return immediately if batch is empty
     final int n = batch.size;

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/gen/vectorization/ExpressionTemplates/FilterStringColumnBetween.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/FilterStringColumnBetween.txt b/ql/src/gen/vectorization/ExpressionTemplates/FilterStringColumnBetween.txt
index b7f70e1..aac8fa8 100644
--- a/ql/src/gen/vectorization/ExpressionTemplates/FilterStringColumnBetween.txt
+++ b/ql/src/gen/vectorization/ExpressionTemplates/FilterStringColumnBetween.txt
@@ -25,7 +25,7 @@ import org.apache.hadoop.hive.ql.exec.vector.expressions.StringExpr;
 import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
 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;
 
 /**
  * This is a generated class to evaluate a [NOT] BETWEEN comparison on a vector of strings.
@@ -55,7 +55,7 @@ public class <ClassName> extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     // return immediately if batch is empty
     final int n = batch.size;

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/gen/vectorization/ExpressionTemplates/FilterStringGroupColumnCompareStringGroupColumn.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/FilterStringGroupColumnCompareStringGroupColumn.txt b/ql/src/gen/vectorization/ExpressionTemplates/FilterStringGroupColumnCompareStringGroupColumn.txt
index 2d18d1d..046ae79 100644
--- a/ql/src/gen/vectorization/ExpressionTemplates/FilterStringGroupColumnCompareStringGroupColumn.txt
+++ b/ql/src/gen/vectorization/ExpressionTemplates/FilterStringGroupColumnCompareStringGroupColumn.txt
@@ -23,6 +23,7 @@ import org.apache.hadoop.hive.ql.exec.vector.expressions.StringExpr;
 import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
 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;
 
 /**
  * Filter the rows in a batch by comparing one string column to another. 
@@ -50,7 +51,7 @@ public class <ClassName> extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     // return immediately if batch is empty
     final int n = batch.size;

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/gen/vectorization/ExpressionTemplates/FilterStringGroupColumnCompareStringGroupScalarBase.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/FilterStringGroupColumnCompareStringGroupScalarBase.txt b/ql/src/gen/vectorization/ExpressionTemplates/FilterStringGroupColumnCompareStringGroupScalarBase.txt
index 76ec8a0..6efa1ca 100644
--- a/ql/src/gen/vectorization/ExpressionTemplates/FilterStringGroupColumnCompareStringGroupScalarBase.txt
+++ b/ql/src/gen/vectorization/ExpressionTemplates/FilterStringGroupColumnCompareStringGroupScalarBase.txt
@@ -25,6 +25,7 @@ import org.apache.hadoop.hive.ql.exec.vector.expressions.StringExpr;
 import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
 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;
 
 /**
  * This is a generated class to evaluate a <OperatorSymbol> comparison on a vector of strings.
@@ -51,7 +52,7 @@ public abstract class <ClassName> extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     // return immediately if batch is empty
     final int n = batch.size;

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/gen/vectorization/ExpressionTemplates/FilterStringGroupScalarCompareStringGroupColumnBase.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/FilterStringGroupScalarCompareStringGroupColumnBase.txt b/ql/src/gen/vectorization/ExpressionTemplates/FilterStringGroupScalarCompareStringGroupColumnBase.txt
index 91d8da5c..7165eb2 100644
--- a/ql/src/gen/vectorization/ExpressionTemplates/FilterStringGroupScalarCompareStringGroupColumnBase.txt
+++ b/ql/src/gen/vectorization/ExpressionTemplates/FilterStringGroupScalarCompareStringGroupColumnBase.txt
@@ -25,6 +25,7 @@ import org.apache.hadoop.hive.ql.exec.vector.expressions.StringExpr;
 import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
 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;
 
 /**
  * This is a generated class to evaluate a <OperatorSymbol> comparison on a vector of strings.
@@ -52,7 +53,7 @@ public abstract class <ClassName> extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     // return immediately if batch is empty
     final int n = batch.size;

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/gen/vectorization/ExpressionTemplates/FilterTimestampColumnBetween.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/FilterTimestampColumnBetween.txt b/ql/src/gen/vectorization/ExpressionTemplates/FilterTimestampColumnBetween.txt
index 604060a..ad5985f 100644
--- a/ql/src/gen/vectorization/ExpressionTemplates/FilterTimestampColumnBetween.txt
+++ b/ql/src/gen/vectorization/ExpressionTemplates/FilterTimestampColumnBetween.txt
@@ -24,6 +24,7 @@ import org.apache.hadoop.hive.ql.exec.vector.TimestampColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
 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;
 
 /**
  * Generated from template FilterTimestampColumnBetween.txt, which covers [NOT] BETWEEN filter
@@ -57,7 +58,7 @@ public class <ClassName> extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     // return immediately if batch is empty
     final int n = batch.size;

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/gen/vectorization/ExpressionTemplates/FilterTimestampColumnCompareLongDoubleColumn.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/FilterTimestampColumnCompareLongDoubleColumn.txt b/ql/src/gen/vectorization/ExpressionTemplates/FilterTimestampColumnCompareLongDoubleColumn.txt
index f9bc9ee..8399334 100644
--- a/ql/src/gen/vectorization/ExpressionTemplates/FilterTimestampColumnCompareLongDoubleColumn.txt
+++ b/ql/src/gen/vectorization/ExpressionTemplates/FilterTimestampColumnCompareLongDoubleColumn.txt
@@ -23,6 +23,7 @@ import org.apache.hadoop.hive.ql.exec.vector.expressions.NullUtil;
 import org.apache.hadoop.hive.ql.exec.vector.*;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 /**
  * Generated from template FilterColumnCompareColumn.txt, which covers binary comparison
@@ -51,7 +52,7 @@ public class <ClassName> extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     // return immediately if batch is empty
     int n = batch.size;

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/gen/vectorization/ExpressionTemplates/FilterTimestampColumnCompareLongDoubleScalar.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/FilterTimestampColumnCompareLongDoubleScalar.txt b/ql/src/gen/vectorization/ExpressionTemplates/FilterTimestampColumnCompareLongDoubleScalar.txt
index fc1be95..57834c2 100644
--- a/ql/src/gen/vectorization/ExpressionTemplates/FilterTimestampColumnCompareLongDoubleScalar.txt
+++ b/ql/src/gen/vectorization/ExpressionTemplates/FilterTimestampColumnCompareLongDoubleScalar.txt
@@ -22,6 +22,7 @@ import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
 import org.apache.hadoop.hive.ql.exec.vector.TimestampColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 /**
  * Generated from template FilterTimestampColumnCompareScalar.txt, which covers comparison
@@ -51,7 +52,7 @@ public class <ClassName> extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     // return immediately if batch is empty
     final int n = batch.size;

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/gen/vectorization/ExpressionTemplates/FilterTimestampColumnCompareTimestampColumn.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/FilterTimestampColumnCompareTimestampColumn.txt b/ql/src/gen/vectorization/ExpressionTemplates/FilterTimestampColumnCompareTimestampColumn.txt
index 0a541f9..b855714 100644
--- a/ql/src/gen/vectorization/ExpressionTemplates/FilterTimestampColumnCompareTimestampColumn.txt
+++ b/ql/src/gen/vectorization/ExpressionTemplates/FilterTimestampColumnCompareTimestampColumn.txt
@@ -27,6 +27,7 @@ import org.apache.hadoop.hive.ql.exec.vector.*;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
 import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 /**
  * Generated from template FilterTimestampColumnCompareColumn.txt, which covers binary comparison
@@ -55,7 +56,7 @@ public class <ClassName> extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     // return immediately if batch is empty
     final int n = batch.size;

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/gen/vectorization/ExpressionTemplates/FilterTimestampColumnCompareTimestampScalar.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/FilterTimestampColumnCompareTimestampScalar.txt b/ql/src/gen/vectorization/ExpressionTemplates/FilterTimestampColumnCompareTimestampScalar.txt
index 68e0006..3fa542b 100644
--- a/ql/src/gen/vectorization/ExpressionTemplates/FilterTimestampColumnCompareTimestampScalar.txt
+++ b/ql/src/gen/vectorization/ExpressionTemplates/FilterTimestampColumnCompareTimestampScalar.txt
@@ -26,6 +26,7 @@ import org.apache.hadoop.hive.ql.exec.vector.TimestampColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.*;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 /**
  * Generated from template FilterColumnCompareScalar.txt, which covers binary comparison
@@ -54,7 +55,7 @@ public class <ClassName> extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     // return immediately if batch is empty
     final int n = batch.size;

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/gen/vectorization/ExpressionTemplates/FilterTimestampScalarCompareTimestampColumn.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/FilterTimestampScalarCompareTimestampColumn.txt b/ql/src/gen/vectorization/ExpressionTemplates/FilterTimestampScalarCompareTimestampColumn.txt
index d5952de..fcda65f 100644
--- a/ql/src/gen/vectorization/ExpressionTemplates/FilterTimestampScalarCompareTimestampColumn.txt
+++ b/ql/src/gen/vectorization/ExpressionTemplates/FilterTimestampScalarCompareTimestampColumn.txt
@@ -28,6 +28,7 @@ import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
 import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
 import org.apache.hadoop.hive.common.type.HiveDecimal;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 /**
  * This is a generated class to evaluate a <OperatorSymbol> comparison on a vector of timestamp
@@ -55,7 +56,7 @@ public class <ClassName> extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     // return immediately if batch is empty
     final int n = batch.size;

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/gen/vectorization/ExpressionTemplates/FilterTruncStringColumnBetween.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/FilterTruncStringColumnBetween.txt b/ql/src/gen/vectorization/ExpressionTemplates/FilterTruncStringColumnBetween.txt
index 44e8e18..416f9e8 100644
--- a/ql/src/gen/vectorization/ExpressionTemplates/FilterTruncStringColumnBetween.txt
+++ b/ql/src/gen/vectorization/ExpressionTemplates/FilterTruncStringColumnBetween.txt
@@ -27,7 +27,7 @@ import org.apache.hadoop.hive.ql.exec.vector.expressions.StringExpr;
 import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
 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;
 
 /**
  * This is a generated class to evaluate a [NOT] BETWEEN comparison on a vector of strings.
@@ -57,7 +57,7 @@ public class <ClassName> extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
      // return immediately if batch is empty
     final int n = batch.size;

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/gen/vectorization/ExpressionTemplates/IfExprColumnScalar.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/IfExprColumnScalar.txt b/ql/src/gen/vectorization/ExpressionTemplates/IfExprColumnScalar.txt
index cf9afa0..d50a6f6 100644
--- a/ql/src/gen/vectorization/ExpressionTemplates/IfExprColumnScalar.txt
+++ b/ql/src/gen/vectorization/ExpressionTemplates/IfExprColumnScalar.txt
@@ -23,6 +23,7 @@ import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.DoubleColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.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.
@@ -56,7 +57,7 @@ public class <ClassName> extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     // return immediately if batch is empty
     final int n = batch.size;

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/gen/vectorization/ExpressionTemplates/IfExprScalarColumn.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/IfExprScalarColumn.txt b/ql/src/gen/vectorization/ExpressionTemplates/IfExprScalarColumn.txt
index 0eb42d1..3658129 100644
--- a/ql/src/gen/vectorization/ExpressionTemplates/IfExprScalarColumn.txt
+++ b/ql/src/gen/vectorization/ExpressionTemplates/IfExprScalarColumn.txt
@@ -23,6 +23,7 @@ import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.DoubleColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.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.
@@ -56,7 +57,7 @@ public class <ClassName> extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     // return immediately if batch is empty
     final int n = batch.size;

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/gen/vectorization/ExpressionTemplates/IfExprScalarScalar.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/IfExprScalarScalar.txt b/ql/src/gen/vectorization/ExpressionTemplates/IfExprScalarScalar.txt
index cec1231..d11459a 100644
--- a/ql/src/gen/vectorization/ExpressionTemplates/IfExprScalarScalar.txt
+++ b/ql/src/gen/vectorization/ExpressionTemplates/IfExprScalarScalar.txt
@@ -23,6 +23,7 @@ import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.DoubleColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 import java.util.Arrays;
 
 /**
@@ -57,7 +58,7 @@ public class <ClassName> extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     // return immediately if batch is empty
     final int n = batch.size;

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/gen/vectorization/ExpressionTemplates/IntervalYearMonthColumnArithmeticDateColumn.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/IntervalYearMonthColumnArithmeticDateColumn.txt b/ql/src/gen/vectorization/ExpressionTemplates/IntervalYearMonthColumnArithmeticDateColumn.txt
index 7c5b614..393413f 100644
--- a/ql/src/gen/vectorization/ExpressionTemplates/IntervalYearMonthColumnArithmeticDateColumn.txt
+++ b/ql/src/gen/vectorization/ExpressionTemplates/IntervalYearMonthColumnArithmeticDateColumn.txt
@@ -27,6 +27,7 @@ import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
 import org.apache.hadoop.hive.ql.util.DateTimeMath;
 import org.apache.hadoop.hive.serde2.io.DateWritable;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 /**
  * Generated from template DateColumnArithmeticIntervalYearMonthColumn.txt, which covers binary arithmetic
@@ -59,7 +60,7 @@ public class <ClassName> extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     // return immediately if batch is empty
     final int n = batch.size;

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/gen/vectorization/ExpressionTemplates/IntervalYearMonthColumnArithmeticDateScalar.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/IntervalYearMonthColumnArithmeticDateScalar.txt b/ql/src/gen/vectorization/ExpressionTemplates/IntervalYearMonthColumnArithmeticDateScalar.txt
index 84d6c4a..a65c4b9 100644
--- a/ql/src/gen/vectorization/ExpressionTemplates/IntervalYearMonthColumnArithmeticDateScalar.txt
+++ b/ql/src/gen/vectorization/ExpressionTemplates/IntervalYearMonthColumnArithmeticDateScalar.txt
@@ -30,6 +30,7 @@ import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
 import org.apache.hadoop.hive.ql.exec.vector.*;
 import org.apache.hadoop.hive.ql.util.DateTimeMath;
 import org.apache.hadoop.hive.serde2.io.DateWritable;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 /**
  * Generated from template DateColumnArithmeticIntervalYearMonthScalar.txt, which covers binary arithmetic
@@ -61,7 +62,7 @@ public class <ClassName> extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     // return immediately if batch is empty
     final int n = batch.size;

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/gen/vectorization/ExpressionTemplates/IntervalYearMonthColumnArithmeticTimestampColumn.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/IntervalYearMonthColumnArithmeticTimestampColumn.txt b/ql/src/gen/vectorization/ExpressionTemplates/IntervalYearMonthColumnArithmeticTimestampColumn.txt
index cfe44c1..08e033a 100644
--- a/ql/src/gen/vectorization/ExpressionTemplates/IntervalYearMonthColumnArithmeticTimestampColumn.txt
+++ b/ql/src/gen/vectorization/ExpressionTemplates/IntervalYearMonthColumnArithmeticTimestampColumn.txt
@@ -27,6 +27,7 @@ import org.apache.hadoop.hive.ql.exec.vector.*;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
 import org.apache.hadoop.hive.ql.util.DateTimeMath;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 /**
  * Generated from template TimestampColumnArithmeticIntervalYearMonthColumn.txt, which covers binary arithmetic
@@ -57,7 +58,7 @@ public class <ClassName> extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     // return immediately if batch is empty
     final int n = batch.size;

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/gen/vectorization/ExpressionTemplates/IntervalYearMonthColumnArithmeticTimestampScalar.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/IntervalYearMonthColumnArithmeticTimestampScalar.txt b/ql/src/gen/vectorization/ExpressionTemplates/IntervalYearMonthColumnArithmeticTimestampScalar.txt
index 22f7abf..c297116 100644
--- a/ql/src/gen/vectorization/ExpressionTemplates/IntervalYearMonthColumnArithmeticTimestampScalar.txt
+++ b/ql/src/gen/vectorization/ExpressionTemplates/IntervalYearMonthColumnArithmeticTimestampScalar.txt
@@ -29,6 +29,7 @@ import org.apache.hadoop.hive.ql.exec.vector.expressions.NullUtil;
 import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
 import org.apache.hadoop.hive.ql.exec.vector.*;
 import org.apache.hadoop.hive.ql.util.DateTimeMath;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 /**
  * Generated from template TimestampColumnArithmeticIntervalYearMonthScalar.txt, which covers binary arithmetic
@@ -59,7 +60,7 @@ public class <ClassName> extends VectorExpression {
   }
 
   @Override
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     // return immediately if batch is empty
     final int n = batch.size;

http://git-wip-us.apache.org/repos/asf/hive/blob/71be5ace/ql/src/gen/vectorization/ExpressionTemplates/IntervalYearMonthScalarArithmeticDateColumn.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/IntervalYearMonthScalarArithmeticDateColumn.txt b/ql/src/gen/vectorization/ExpressionTemplates/IntervalYearMonthScalarArithmeticDateColumn.txt
index ffc2cec..34d516e 100644
--- a/ql/src/gen/vectorization/ExpressionTemplates/IntervalYearMonthScalarArithmeticDateColumn.txt
+++ b/ql/src/gen/vectorization/ExpressionTemplates/IntervalYearMonthScalarArithmeticDateColumn.txt
@@ -38,6 +38,7 @@ import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.NullUtil;
 import org.apache.hadoop.hive.ql.util.DateTimeMath;
 import org.apache.hadoop.hive.serde2.io.DateWritable;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 /**
  * Generated from template DateTimeScalarArithmeticIntervalYearMonthColumn.txt.
@@ -75,7 +76,7 @@ public class <ClassName> extends VectorExpression {
    *
    * @batch a package of rows with each column stored in a vector
    */
-  public void evaluate(VectorizedRowBatch batch) {
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
 
     // return immediately if batch is empty
     final int n = batch.size;


[40/50] [abbrv] hive git commit: HIVE-19381: Function replication in cloud fail when download resource from AWS (Daniel Dai, reviewed by Thejas Nair)

Posted by vg...@apache.org.
HIVE-19381: Function replication in cloud fail when download resource from AWS (Daniel Dai, reviewed by Thejas Nair)


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

Branch: refs/heads/branch-3.0.0
Commit: a9105ced0c7d6684e590f536f2299435076d29c9
Parents: 6936c9c
Author: Daniel Dai <da...@gmail.com>
Authored: Mon May 14 13:25:58 2018 -0700
Committer: Daniel Dai <da...@gmail.com>
Committed: Mon May 14 13:25:58 2018 -0700

----------------------------------------------------------------------
 ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionTask.java   | 6 ++++++
 .../java/org/apache/hadoop/hive/ql/session/SessionState.java   | 5 ++++-
 2 files changed, 10 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/a9105ced/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionTask.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionTask.java
index a0a90a9..b9d6f58 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionTask.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionTask.java
@@ -30,6 +30,7 @@ import com.google.common.collect.Multimap;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.api.Function;
 import org.apache.hadoop.hive.metastore.api.PrincipalType;
 import org.apache.hadoop.hive.metastore.api.ResourceType;
@@ -165,7 +166,9 @@ public class FunctionTask extends Task<FunctionWork> {
     checkLocalFunctionResources(db, createFunctionDesc.getResources());
 
     FunctionInfo registered = null;
+    HiveConf oldConf = SessionState.get().getConf();
     try {
+      SessionState.get().setConf(conf);
       registered = FunctionRegistry.registerPermanentFunction(
         registeredName, className, true, toFunctionResource(resources));
     } catch (RuntimeException ex) {
@@ -173,7 +176,10 @@ public class FunctionTask extends Task<FunctionWork> {
       while (t.getCause() != null) {
         t = t.getCause();
       }
+    } finally {
+      SessionState.get().setConf(oldConf);
     }
+
     if (registered == null) {
       console.printError("Failed to register " + registeredName
           + " using class " + createFunctionDesc.getClassName());

http://git-wip-us.apache.org/repos/asf/hive/blob/a9105ced/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java b/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java
index 6bb756c..89129f9 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java
@@ -135,7 +135,7 @@ public class SessionState {
   /**
    * current configuration.
    */
-  private final HiveConf sessionConf;
+  private HiveConf sessionConf;
 
   /**
    * silent mode.
@@ -308,6 +308,9 @@ public class SessionState {
     return sessionConf;
   }
 
+  public void setConf(HiveConf conf) {
+    this.sessionConf = conf;
+  }
 
   public File getTmpOutputFile() {
     return tmpOutputFile;


[44/50] [abbrv] hive git commit: HIVE-19534: Allow implementations to access member variables of AbstractRecordWriter (Prasanth Jayachandran reviewed by Matt Burgess, Ashutosh Chauhan)

Posted by vg...@apache.org.
HIVE-19534: Allow implementations to access member variables of AbstractRecordWriter (Prasanth Jayachandran reviewed by Matt Burgess, Ashutosh Chauhan)


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

Branch: refs/heads/branch-3.0.0
Commit: 3ea0356f7dd9fc4d3406806d80c349187afd9d64
Parents: 66f6748
Author: Prasanth Jayachandran <pr...@apache.org>
Authored: Mon May 14 17:19:34 2018 -0700
Committer: Prasanth Jayachandran <pr...@apache.org>
Committed: Mon May 14 17:19:34 2018 -0700

----------------------------------------------------------------------
 .../hive/streaming/AbstractRecordWriter.java    | 92 ++++++++++----------
 1 file changed, 46 insertions(+), 46 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/3ea0356f/streaming/src/java/org/apache/hive/streaming/AbstractRecordWriter.java
----------------------------------------------------------------------
diff --git a/streaming/src/java/org/apache/hive/streaming/AbstractRecordWriter.java b/streaming/src/java/org/apache/hive/streaming/AbstractRecordWriter.java
index b6c8890..0866850 100644
--- a/streaming/src/java/org/apache/hive/streaming/AbstractRecordWriter.java
+++ b/streaming/src/java/org/apache/hive/streaming/AbstractRecordWriter.java
@@ -64,48 +64,48 @@ public abstract class AbstractRecordWriter implements RecordWriter {
 
   private static final String DEFAULT_LINE_DELIMITER_PATTERN = "[\r\n]";
   protected HiveConf conf;
-  private StreamingConnection conn;
+  protected StreamingConnection conn;
   protected Table table;
-  List<String> inputColumns;
-  List<String> inputTypes;
-  private String fullyQualifiedTableName;
-  private Map<String, List<RecordUpdater>> updaters = new HashMap<>();
-  private Map<String, Path> partitionPaths = new HashMap<>();
-  private Set<String> addedPartitions = new HashSet<>();
+  protected List<String> inputColumns;
+  protected List<String> inputTypes;
+  protected String fullyQualifiedTableName;
+  protected Map<String, List<RecordUpdater>> updaters = new HashMap<>();
+  protected Map<String, Path> partitionPaths = new HashMap<>();
+  protected Set<String> addedPartitions = new HashSet<>();
   // input OI includes table columns + partition columns
-  private StructObjectInspector inputRowObjectInspector;
+  protected StructObjectInspector inputRowObjectInspector;
   // output OI strips off the partition columns and retains other columns
-  private ObjectInspector outputRowObjectInspector;
-  private List<String> partitionColumns = new ArrayList<>();
-  private ObjectInspector[] partitionObjInspectors = null;
-  private StructField[] partitionStructFields = null;
-  private Object[] partitionFieldData;
-  private ObjectInspector[] bucketObjInspectors = null;
-  private StructField[] bucketStructFields = null;
-  private Object[] bucketFieldData;
-  private List<Integer> bucketIds = new ArrayList<>();
-  private int totalBuckets;
-  private String defaultPartitionName;
-  private boolean isBucketed;
-  private AcidOutputFormat<?, ?> acidOutputFormat;
-  private Long curBatchMinWriteId;
-  private Long curBatchMaxWriteId;
-  private final String lineDelimiter;
-  private HeapMemoryMonitor heapMemoryMonitor;
+  protected ObjectInspector outputRowObjectInspector;
+  protected List<String> partitionColumns = new ArrayList<>();
+  protected ObjectInspector[] partitionObjInspectors = null;
+  protected StructField[] partitionStructFields = null;
+  protected Object[] partitionFieldData;
+  protected ObjectInspector[] bucketObjInspectors = null;
+  protected StructField[] bucketStructFields = null;
+  protected Object[] bucketFieldData;
+  protected List<Integer> bucketIds = new ArrayList<>();
+  protected int totalBuckets;
+  protected String defaultPartitionName;
+  protected boolean isBucketed;
+  protected AcidOutputFormat<?, ?> acidOutputFormat;
+  protected Long curBatchMinWriteId;
+  protected Long curBatchMaxWriteId;
+  protected final String lineDelimiter;
+  protected HeapMemoryMonitor heapMemoryMonitor;
   // if low memory canary is set and if records after set canary exceeds threshold, trigger a flush.
   // This is to avoid getting notified of low memory too often and flushing too often.
-  private AtomicBoolean lowMemoryCanary;
-  private long ingestSizeBytes = 0;
-  private boolean autoFlush;
-  private float memoryUsageThreshold;
-  private long ingestSizeThreshold;
+  protected AtomicBoolean lowMemoryCanary;
+  protected long ingestSizeBytes = 0;
+  protected boolean autoFlush;
+  protected float memoryUsageThreshold;
+  protected long ingestSizeThreshold;
 
   public AbstractRecordWriter(final String lineDelimiter) {
     this.lineDelimiter = lineDelimiter == null || lineDelimiter.isEmpty() ?
       DEFAULT_LINE_DELIMITER_PATTERN : lineDelimiter;
   }
 
-  private static class OrcMemoryPressureMonitor implements HeapMemoryMonitor.Listener {
+  protected static class OrcMemoryPressureMonitor implements HeapMemoryMonitor.Listener {
     private static final Logger LOG = LoggerFactory.getLogger(OrcMemoryPressureMonitor.class.getName());
     private final AtomicBoolean lowMemoryCanary;
 
@@ -179,7 +179,7 @@ public abstract class AbstractRecordWriter implements RecordWriter {
     }
   }
 
-  private void setupMemoryMonitoring() {
+  protected void setupMemoryMonitoring() {
     this.autoFlush = conf.getBoolVar(HiveConf.ConfVars.HIVE_STREAMING_AUTO_FLUSH_ENABLED);
     this.memoryUsageThreshold = conf.getFloatVar(HiveConf.ConfVars.HIVE_HEAP_MEMORY_MONITOR_USAGE_THRESHOLD);
     this.ingestSizeThreshold = conf.getSizeVar(HiveConf.ConfVars.HIVE_STREAMING_AUTO_FLUSH_CHECK_INTERVAL_SIZE);
@@ -201,7 +201,7 @@ public abstract class AbstractRecordWriter implements RecordWriter {
     }
   }
 
-  private void prepareBucketingFields() {
+  protected void prepareBucketingFields() {
     this.isBucketed = table.getSd().getNumBuckets() > 0;
     // For unbucketed tables we have exactly 1 RecordUpdater (until HIVE-19208) for each AbstractRecordWriter which
     // ends up writing to a file bucket_000000.
@@ -219,7 +219,7 @@ public abstract class AbstractRecordWriter implements RecordWriter {
     }
   }
 
-  private void preparePartitioningFields() {
+  protected void preparePartitioningFields() {
     final int numPartitions = table.getPartitionKeys().size();
     this.partitionFieldData = new Object[numPartitions];
     this.partitionObjInspectors = new ObjectInspector[numPartitions];
@@ -240,12 +240,12 @@ public abstract class AbstractRecordWriter implements RecordWriter {
   /**
    * used to tag error msgs to provided some breadcrumbs
    */
-  private String getWatermark(String partition) {
+  protected String getWatermark(String partition) {
     return partition + " writeIds[" + curBatchMinWriteId + "," + curBatchMaxWriteId + "]";
   }
 
   // return the column numbers of the bucketed columns
-  private List<Integer> getBucketColIDs(List<String> bucketCols, List<FieldSchema> cols) {
+  protected List<Integer> getBucketColIDs(List<String> bucketCols, List<FieldSchema> cols) {
     ArrayList<Integer> result = new ArrayList<>(bucketCols.size());
     HashSet<String> bucketSet = new HashSet<>(bucketCols);
     for (int i = 0; i < cols.size(); i++) {
@@ -275,7 +275,7 @@ public abstract class AbstractRecordWriter implements RecordWriter {
   public abstract Object encode(byte[] record) throws SerializationError;
 
   // returns the bucket number to which the record belongs to
-  private int getBucket(Object row) {
+  protected int getBucket(Object row) {
     if (!isBucketed) {
       return 0;
     }
@@ -288,7 +288,7 @@ public abstract class AbstractRecordWriter implements RecordWriter {
       ObjectInspectorUtils.getBucketNumberOld(bucketFields, bucketObjInspectors, totalBuckets);
   }
 
-  private List<String> getPartitionValues(final Object row) {
+  protected List<String> getPartitionValues(final Object row) {
     if (!conn.isPartitionedTable()) {
       return null;
     }
@@ -359,7 +359,7 @@ public abstract class AbstractRecordWriter implements RecordWriter {
     }
   }
 
-  private static ObjectInspector[] getObjectInspectorsForBucketedCols(List<Integer> bucketIds
+  protected static ObjectInspector[] getObjectInspectorsForBucketedCols(List<Integer> bucketIds
     , StructObjectInspector recordObjInspector) {
     ObjectInspector[] result = new ObjectInspector[bucketIds.size()];
 
@@ -371,14 +371,14 @@ public abstract class AbstractRecordWriter implements RecordWriter {
     return result;
   }
 
-  private Object[] getBucketFields(Object row) {
+  protected Object[] getBucketFields(Object row) {
     for (int i = 0; i < bucketIds.size(); i++) {
       bucketFieldData[i] = inputRowObjectInspector.getStructFieldData(row, bucketStructFields[i]);
     }
     return bucketFieldData;
   }
 
-  private Object[] getPartitionFields(Object row) {
+  protected Object[] getPartitionFields(Object row) {
     for (int i = 0; i < partitionFieldData.length; i++) {
       partitionFieldData[i] = inputRowObjectInspector.getStructFieldData(row, partitionStructFields[i]);
     }
@@ -412,7 +412,7 @@ public abstract class AbstractRecordWriter implements RecordWriter {
     }
   }
 
-  private void checkAutoFlush() throws StreamingIOFailure {
+  protected void checkAutoFlush() throws StreamingIOFailure {
     if (!autoFlush) {
       return;
     }
@@ -444,7 +444,7 @@ public abstract class AbstractRecordWriter implements RecordWriter {
     return addedPartitions;
   }
 
-  private RecordUpdater createRecordUpdater(final Path partitionPath, int bucketId, Long minWriteId,
+  protected RecordUpdater createRecordUpdater(final Path partitionPath, int bucketId, Long minWriteId,
     Long maxWriteID)
     throws IOException {
     // Initialize table properties from the table parameters. This is required because the table
@@ -463,7 +463,7 @@ public abstract class AbstractRecordWriter implements RecordWriter {
         .finalDestination(partitionPath));
   }
 
-  private RecordUpdater getRecordUpdater(List<String> partitionValues, int bucketId) throws StreamingIOFailure {
+  protected RecordUpdater getRecordUpdater(List<String> partitionValues, int bucketId) throws StreamingIOFailure {
     RecordUpdater recordUpdater;
     String key;
     Path destLocation;
@@ -510,7 +510,7 @@ public abstract class AbstractRecordWriter implements RecordWriter {
     return recordUpdater;
   }
 
-  private List<RecordUpdater> initializeBuckets() {
+  protected List<RecordUpdater> initializeBuckets() {
     List<RecordUpdater> result = new ArrayList<>(totalBuckets);
     for (int bucket = 0; bucket < totalBuckets; bucket++) {
       result.add(bucket, null); //so that get(i) returns null rather than ArrayOutOfBounds
@@ -518,7 +518,7 @@ public abstract class AbstractRecordWriter implements RecordWriter {
     return result;
   }
 
-  private void logStats(final String prefix) {
+  protected void logStats(final String prefix) {
     int openRecordUpdaters = updaters.values()
       .stream()
       .mapToInt(List::size)


[23/50] [abbrv] hive git commit: HIVE-19435: Incremental replication cause data loss if a table is dropped followed by create and insert-into with different partition type (Sankar Hariappan, reviewed by Mahesh Kumar Behera, Thejas M Nair)

Posted by vg...@apache.org.
HIVE-19435: Incremental replication cause data loss if a table is dropped followed by create and insert-into with different partition type (Sankar Hariappan, reviewed by Mahesh Kumar Behera, Thejas M Nair)


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

Branch: refs/heads/branch-3.0.0
Commit: 91b66c5c5a6106d42903385b7a49e6b32400cc39
Parents: 11a7164
Author: Sankar Hariappan <sa...@apache.org>
Authored: Sun May 13 13:02:25 2018 +0530
Committer: Sankar Hariappan <sa...@apache.org>
Committed: Sun May 13 13:02:25 2018 +0530

----------------------------------------------------------------------
 ...TestReplicationScenariosAcrossInstances.java | 58 ++++++++++++++++++++
 .../hive/ql/parse/ImportSemanticAnalyzer.java   | 38 ++++++++++---
 2 files changed, 88 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/91b66c5c/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenariosAcrossInstances.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenariosAcrossInstances.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenariosAcrossInstances.java
index 70e1aa7..df9bde0 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenariosAcrossInstances.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenariosAcrossInstances.java
@@ -675,4 +675,62 @@ public class TestReplicationScenariosAcrossInstances {
             .run("select id from table2 order by id")
             .verifyResults(new String[] {"2"});
   }
+
+  @Test
+  public void testIncrementalReplWithDropAndCreateTableDifferentPartitionTypeAndInsert() throws Throwable {
+    // Bootstrap dump with empty db
+    WarehouseInstance.Tuple bootstrapTuple = primary.dump(primaryDbName, null);
+
+    // Bootstrap load in replica
+    replica.load(replicatedDbName, bootstrapTuple.dumpLocation)
+            .status(replicatedDbName)
+            .verifyResult(bootstrapTuple.lastReplicationId);
+
+    // First incremental dump
+    WarehouseInstance.Tuple firstIncremental = primary.run("use " + primaryDbName)
+            .run("create table table1 (id int) partitioned by (country string)")
+            .run("create table table2 (id int)")
+            .run("create table table3 (id int) partitioned by (country string)")
+            .run("insert into table1 partition(country='india') values(1)")
+            .run("insert into table2 values(2)")
+            .run("insert into table3 partition(country='india') values(3)")
+            .dump(primaryDbName, bootstrapTuple.lastReplicationId);
+
+    // Second incremental dump
+    WarehouseInstance.Tuple secondIncremental = primary.run("use " + primaryDbName)
+            .run("drop table table1")
+            .run("drop table table2")
+            .run("drop table table3")
+            .run("create table table1 (id int)")
+            .run("insert into table1 values (10)")
+            .run("create table table2 (id int) partitioned by (country string)")
+            .run("insert into table2 partition(country='india') values(20)")
+            .run("create table table3 (id int) partitioned by (name string, rank int)")
+            .run("insert into table3 partition(name='adam', rank=100) values(30)")
+            .dump(primaryDbName, firstIncremental.lastReplicationId);
+
+    // First incremental load
+    replica.load(replicatedDbName, firstIncremental.dumpLocation)
+            .status(replicatedDbName)
+            .verifyResult(firstIncremental.lastReplicationId)
+            .run("use " + replicatedDbName)
+            .run("select id from table1")
+            .verifyResults(new String[] {"1"})
+            .run("select * from table2")
+            .verifyResults(new String[] {"2"})
+            .run("select id from table3")
+            .verifyResults(new String[] {"3"});
+
+    // Second incremental load
+    replica.load(replicatedDbName, secondIncremental.dumpLocation)
+            .status(replicatedDbName)
+            .verifyResult(secondIncremental.lastReplicationId)
+            .run("use " + replicatedDbName)
+            .run("select * from table1")
+            .verifyResults(new String[] {"10"})
+            .run("select id from table2")
+            .verifyResults(new String[] {"20"})
+            .run("select id from table3")
+            .verifyResults(new String[] {"30"});
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/91b66c5c/ql/src/java/org/apache/hadoop/hive/ql/parse/ImportSemanticAnalyzer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/ImportSemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/ImportSemanticAnalyzer.java
index e6a7012..eb6708b 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/ImportSemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/ImportSemanticAnalyzer.java
@@ -898,14 +898,14 @@ public class ImportSemanticAnalyzer extends BaseSemanticAnalyzer {
     }
   }
 
-  private static Table createNewTableMetadataObject(ImportTableDesc tblDesk)
+  private static Table createNewTableMetadataObject(ImportTableDesc tblDesc)
       throws SemanticException {
-    Table newTable = new Table(tblDesk.getDatabaseName(), tblDesk.getTableName());
+    Table newTable = new Table(tblDesc.getDatabaseName(), tblDesc.getTableName());
     //so that we know the type of table we are creating: acid/MM to match what was exported
-    newTable.setParameters(tblDesk.getTblProps());
-    if(tblDesk.isExternal() && AcidUtils.isTransactionalTable(newTable)) {
+    newTable.setParameters(tblDesc.getTblProps());
+    if(tblDesc.isExternal() && AcidUtils.isTransactionalTable(newTable)) {
       throw new SemanticException("External tables may not be transactional: " +
-          Warehouse.getQualifiedName(tblDesk.getDatabaseName(), tblDesk.getTableName()));
+          Warehouse.getQualifiedName(tblDesc.getDatabaseName(), tblDesc.getTableName()));
     }
     return newTable;
   }
@@ -1027,14 +1027,36 @@ public class ImportSemanticAnalyzer extends BaseSemanticAnalyzer {
         x.getTasks().add(t);
       }
     } else {
+      // If table of current event has partition flag different from existing table, it means, some
+      // of the previous events in same batch have drop and create table events with same same but
+      // different partition flag. In this case, should go with current event's table type and so
+      // create the dummy table object for adding repl tasks.
+      boolean isOldTableValid = true;
+      if (table.isPartitioned() != isPartitioned(tblDesc)) {
+        table = createNewTableMetadataObject(tblDesc);
+        isOldTableValid = false;
+      }
+
       // Table existed, and is okay to replicate into, not dropping and re-creating.
-      if (table.isPartitioned()) {
+      if (isPartitioned(tblDesc)) {
         x.getLOG().debug("table partitioned");
         for (AddPartitionDesc addPartitionDesc : partitionDescs) {
           addPartitionDesc.setReplicationSpec(replicationSpec);
           Map<String, String> partSpec = addPartitionDesc.getPartition(0).getPartSpec();
           org.apache.hadoop.hive.ql.metadata.Partition ptn = null;
-          if ((ptn = x.getHive().getPartition(table, partSpec, false)) == null) {
+          if (isOldTableValid) {
+            // If existing table is valid but the partition spec is different, then ignore partition
+            // validation and create new partition.
+            try {
+              ptn = x.getHive().getPartition(table, partSpec, false);
+            } catch (HiveException ex) {
+              ptn = null;
+              table = createNewTableMetadataObject(tblDesc);
+              isOldTableValid = false;
+            }
+          }
+
+          if (ptn == null) {
             if (!replicationSpec.isMetadataOnly()){
               x.getTasks().add(addSinglePartition(
                   fromURI, fs, tblDesc, table, wh, addPartitionDesc, replicationSpec, x, writeId, stmtId, isSourceMm));
@@ -1079,7 +1101,7 @@ public class ImportSemanticAnalyzer extends BaseSemanticAnalyzer {
         x.getLOG().debug("table non-partitioned");
         if (!replicationSpec.isMetadataOnly()) {
           // repl-imports are replace-into unless the event is insert-into
-          loadTable(fromURI, table, replicationSpec.isReplace(), table.getDataLocation(),
+          loadTable(fromURI, table, replicationSpec.isReplace(), new Path(tblDesc.getLocation()),
             replicationSpec, x, writeId, stmtId, isSourceMm);
         } else {
           x.getTasks().add(alterTableTask(tblDesc, x, replicationSpec));


[07/50] [abbrv] hive git commit: Revert "HIVE-19310 : Metastore: MetaStoreDirectSql.ensureDbInit has some slow DN calls which might need to be run only in test env (Sergey Shelukhin, reviewed by Gopal Vijayaraghavan)"

Posted by vg...@apache.org.
Revert "HIVE-19310 : Metastore: MetaStoreDirectSql.ensureDbInit has some slow DN calls which might need to be run only in test env (Sergey Shelukhin, reviewed by Gopal Vijayaraghavan)"

This reverts commit 41f570a2c303cf390806e4681c73873486fdb19d.


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

Branch: refs/heads/branch-3.0.0
Commit: 71d77ab1b268b200c93cb9e69f2f2ee74afa8c96
Parents: d5d8982
Author: Vineet Garg <vg...@apache.org>
Authored: Tue May 8 15:41:30 2018 -0700
Committer: Vineet Garg <vg...@apache.org>
Committed: Tue May 8 15:41:30 2018 -0700

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hive/metastore/MetaStoreDirectSql.java | 3 +--
 1 file changed, 1 insertion(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/71d77ab1/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreDirectSql.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreDirectSql.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreDirectSql.java
index e2ca6d2..997f5fd 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreDirectSql.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreDirectSql.java
@@ -179,8 +179,7 @@ class MetaStoreDirectSql {
           + " Disabling directSQL as it uses hand-hardcoded SQL with that assumption.");
       isCompatibleDatastore = false;
     } else {
-      boolean isInTest = MetastoreConf.getBoolVar(conf, ConfVars.HIVE_IN_TEST);
-      isCompatibleDatastore = (!isInTest || ensureDbInit()) && runTestQuery();
+      isCompatibleDatastore = ensureDbInit() && runTestQuery();
       if (isCompatibleDatastore) {
         LOG.info("Using direct SQL, underlying DB is " + dbType);
       }


[12/50] [abbrv] hive git commit: HIVE-19451: Druid Query Execution fails with ClassNotFoundException org.antlr.v4.runtime.CharStream (Nishant Bangarwa reviewed by Jesus Camacho Rodriguez)

Posted by vg...@apache.org.
HIVE-19451: Druid Query Execution fails with ClassNotFoundException org.antlr.v4.runtime.CharStream (Nishant Bangarwa reviewed by Jesus Camacho Rodriguez)


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

Branch: refs/heads/branch-3.0.0
Commit: 31027371f9040cb51c3e680c19c8100ef3415b4f
Parents: 78e6bfa
Author: Nishant Bangarwa <ni...@gmail.com>
Authored: Wed May 9 09:35:25 2018 -0700
Committer: Jesus Camacho Rodriguez <jc...@apache.org>
Committed: Wed May 9 09:36:06 2018 -0700

----------------------------------------------------------------------
 druid-handler/pom.xml | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/31027371/druid-handler/pom.xml
----------------------------------------------------------------------
diff --git a/druid-handler/pom.xml b/druid-handler/pom.xml
index d25706c..9c2bebf 100644
--- a/druid-handler/pom.xml
+++ b/druid-handler/pom.xml
@@ -362,6 +362,7 @@
                       <include>org.roaringbitmap:*</include>
                       <include>org.apache.derby:*</include>
                       <include>org.asynchttpclient:*</include>
+                      <include>org.antlr:*</include>
                     </includes>
                   </artifactSet>
                   <filters>


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

Posted by vg...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/e2d4f347/ql/src/test/results/clientpositive/spark/vectorized_timestamp_funcs.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/vectorized_timestamp_funcs.q.out b/ql/src/test/results/clientpositive/spark/vectorized_timestamp_funcs.q.out
index 68b89a7..fe5fd23 100644
--- a/ql/src/test/results/clientpositive/spark/vectorized_timestamp_funcs.q.out
+++ b/ql/src/test/results/clientpositive/spark/vectorized_timestamp_funcs.q.out
@@ -1,15 +1,19 @@
-PREHOOK: query: CREATE TABLE alltypesorc_string(ctimestamp1 timestamp, stimestamp1 string) STORED AS ORC
+PREHOOK: query: CREATE TABLE alltypesorc_string(cboolean1 boolean, ctimestamp1 timestamp, stimestamp1 string,
+    ctimestamp2 timestamp) STORED AS ORC
 PREHOOK: type: CREATETABLE
 PREHOOK: Output: database:default
 PREHOOK: Output: default@alltypesorc_string
-POSTHOOK: query: CREATE TABLE alltypesorc_string(ctimestamp1 timestamp, stimestamp1 string) STORED AS ORC
+POSTHOOK: query: CREATE TABLE alltypesorc_string(cboolean1 boolean, ctimestamp1 timestamp, stimestamp1 string,
+    ctimestamp2 timestamp) STORED AS ORC
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@alltypesorc_string
 PREHOOK: query: INSERT OVERWRITE TABLE alltypesorc_string
 SELECT
+  cboolean1,
   to_utc_timestamp(ctimestamp1, 'America/Los_Angeles') AS toutc,
-  CAST(to_utc_timestamp(ctimestamp1, 'America/Los_Angeles') AS STRING) as cst
+  CAST(to_utc_timestamp(ctimestamp1, 'America/Los_Angeles') AS STRING) as cst,
+  ctimestamp2
 FROM alltypesorc
 ORDER BY toutc, cst
 LIMIT 40
@@ -18,16 +22,146 @@ PREHOOK: Input: default@alltypesorc
 PREHOOK: Output: default@alltypesorc_string
 POSTHOOK: query: INSERT OVERWRITE TABLE alltypesorc_string
 SELECT
+  cboolean1,
   to_utc_timestamp(ctimestamp1, 'America/Los_Angeles') AS toutc,
-  CAST(to_utc_timestamp(ctimestamp1, 'America/Los_Angeles') AS STRING) as cst
+  CAST(to_utc_timestamp(ctimestamp1, 'America/Los_Angeles') AS STRING) as cst,
+  ctimestamp2
 FROM alltypesorc
 ORDER BY toutc, cst
 LIMIT 40
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@alltypesorc
 POSTHOOK: Output: default@alltypesorc_string
+POSTHOOK: Lineage: alltypesorc_string.cboolean1 SIMPLE [(alltypesorc)alltypesorc.FieldSchema(name:cboolean1, type:boolean, comment:null), ]
 POSTHOOK: Lineage: alltypesorc_string.ctimestamp1 EXPRESSION [(alltypesorc)alltypesorc.FieldSchema(name:ctimestamp1, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: alltypesorc_string.ctimestamp2 SIMPLE [(alltypesorc)alltypesorc.FieldSchema(name:ctimestamp2, type:timestamp, comment:null), ]
 POSTHOOK: Lineage: alltypesorc_string.stimestamp1 EXPRESSION [(alltypesorc)alltypesorc.FieldSchema(name:ctimestamp1, type:timestamp, comment:null), ]
+PREHOOK: query: INSERT INTO TABLE alltypesorc_string values (false, '2021-09-24 03:18:32.4', '1978-08-05 14:41:05.501', '1999-10-03 16:59:10.396903939')
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@alltypesorc_string
+POSTHOOK: query: INSERT INTO TABLE alltypesorc_string values (false, '2021-09-24 03:18:32.4', '1978-08-05 14:41:05.501', '1999-10-03 16:59:10.396903939')
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@alltypesorc_string
+POSTHOOK: Lineage: alltypesorc_string.cboolean1 SCRIPT []
+POSTHOOK: Lineage: alltypesorc_string.ctimestamp1 SCRIPT []
+POSTHOOK: Lineage: alltypesorc_string.ctimestamp2 SCRIPT []
+POSTHOOK: Lineage: alltypesorc_string.stimestamp1 SCRIPT []
+PREHOOK: query: INSERT INTO TABLE alltypesorc_string values (false, null, '2013-04-10 00:43:46.8547315', null)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@alltypesorc_string
+POSTHOOK: query: INSERT INTO TABLE alltypesorc_string values (false, null, '2013-04-10 00:43:46.8547315', null)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@alltypesorc_string
+POSTHOOK: Lineage: alltypesorc_string.cboolean1 SCRIPT []
+POSTHOOK: Lineage: alltypesorc_string.ctimestamp1 EXPRESSION []
+POSTHOOK: Lineage: alltypesorc_string.ctimestamp2 EXPRESSION []
+POSTHOOK: Lineage: alltypesorc_string.stimestamp1 SCRIPT []
+PREHOOK: query: INSERT INTO TABLE alltypesorc_string values (false, '2021-09-24 03:18:32.4', null, null)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@alltypesorc_string
+POSTHOOK: query: INSERT INTO TABLE alltypesorc_string values (false, '2021-09-24 03:18:32.4', null, null)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@alltypesorc_string
+POSTHOOK: Lineage: alltypesorc_string.cboolean1 SCRIPT []
+POSTHOOK: Lineage: alltypesorc_string.ctimestamp1 SCRIPT []
+POSTHOOK: Lineage: alltypesorc_string.ctimestamp2 EXPRESSION []
+POSTHOOK: Lineage: alltypesorc_string.stimestamp1 EXPRESSION []
+PREHOOK: query: INSERT INTO TABLE alltypesorc_string values (null, '7160-12-02 06:00:24.81200852', '0004-09-22 18:26:29.519542222', '1966-08-16 13:36:50.183')
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@alltypesorc_string
+POSTHOOK: query: INSERT INTO TABLE alltypesorc_string values (null, '7160-12-02 06:00:24.81200852', '0004-09-22 18:26:29.519542222', '1966-08-16 13:36:50.183')
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@alltypesorc_string
+POSTHOOK: Lineage: alltypesorc_string.cboolean1 EXPRESSION []
+POSTHOOK: Lineage: alltypesorc_string.ctimestamp1 SCRIPT []
+POSTHOOK: Lineage: alltypesorc_string.ctimestamp2 SCRIPT []
+POSTHOOK: Lineage: alltypesorc_string.stimestamp1 SCRIPT []
+PREHOOK: query: INSERT INTO TABLE alltypesorc_string values (null, null, '4966-12-04 09:30:55.202', null)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@alltypesorc_string
+POSTHOOK: query: INSERT INTO TABLE alltypesorc_string values (null, null, '4966-12-04 09:30:55.202', null)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@alltypesorc_string
+POSTHOOK: Lineage: alltypesorc_string.cboolean1 EXPRESSION []
+POSTHOOK: Lineage: alltypesorc_string.ctimestamp1 EXPRESSION []
+POSTHOOK: Lineage: alltypesorc_string.ctimestamp2 EXPRESSION []
+POSTHOOK: Lineage: alltypesorc_string.stimestamp1 SCRIPT []
+PREHOOK: query: INSERT INTO TABLE alltypesorc_string values (null, '7160-12-02 06:00:24.81200852', null, null)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@alltypesorc_string
+POSTHOOK: query: INSERT INTO TABLE alltypesorc_string values (null, '7160-12-02 06:00:24.81200852', null, null)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@alltypesorc_string
+POSTHOOK: Lineage: alltypesorc_string.cboolean1 EXPRESSION []
+POSTHOOK: Lineage: alltypesorc_string.ctimestamp1 SCRIPT []
+POSTHOOK: Lineage: alltypesorc_string.ctimestamp2 EXPRESSION []
+POSTHOOK: Lineage: alltypesorc_string.stimestamp1 EXPRESSION []
+PREHOOK: query: INSERT INTO TABLE alltypesorc_string values (true, '1985-07-20 09:30:11.0', '8521-01-16 20:42:05.668832', '1319-02-02 16:31:57.778')
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@alltypesorc_string
+POSTHOOK: query: INSERT INTO TABLE alltypesorc_string values (true, '1985-07-20 09:30:11.0', '8521-01-16 20:42:05.668832', '1319-02-02 16:31:57.778')
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@alltypesorc_string
+POSTHOOK: Lineage: alltypesorc_string.cboolean1 SCRIPT []
+POSTHOOK: Lineage: alltypesorc_string.ctimestamp1 SCRIPT []
+POSTHOOK: Lineage: alltypesorc_string.ctimestamp2 SCRIPT []
+POSTHOOK: Lineage: alltypesorc_string.stimestamp1 SCRIPT []
+PREHOOK: query: INSERT INTO TABLE alltypesorc_string values (true, null, '1883-04-17 04:14:34.64776', '2024-11-11 16:42:41.101')
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@alltypesorc_string
+POSTHOOK: query: INSERT INTO TABLE alltypesorc_string values (true, null, '1883-04-17 04:14:34.64776', '2024-11-11 16:42:41.101')
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@alltypesorc_string
+POSTHOOK: Lineage: alltypesorc_string.cboolean1 SCRIPT []
+POSTHOOK: Lineage: alltypesorc_string.ctimestamp1 EXPRESSION []
+POSTHOOK: Lineage: alltypesorc_string.ctimestamp2 SCRIPT []
+POSTHOOK: Lineage: alltypesorc_string.stimestamp1 SCRIPT []
+PREHOOK: query: INSERT INTO TABLE alltypesorc_string values (true, '0528-10-27 08:15:18.941718273', null, null)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@alltypesorc_string
+POSTHOOK: query: INSERT INTO TABLE alltypesorc_string values (true, '0528-10-27 08:15:18.941718273', null, null)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@alltypesorc_string
+POSTHOOK: Lineage: alltypesorc_string.cboolean1 SCRIPT []
+POSTHOOK: Lineage: alltypesorc_string.ctimestamp1 SCRIPT []
+POSTHOOK: Lineage: alltypesorc_string.ctimestamp2 EXPRESSION []
+POSTHOOK: Lineage: alltypesorc_string.stimestamp1 EXPRESSION []
+PREHOOK: query: INSERT INTO TABLE alltypesorc_string values
+     (false, '2021-09-24 03:18:32.4', '1985-11-18 16:37:54.0', '2010-04-08 02:43:35.861742727'),
+     (true, null, '1985-11-18 16:37:54.0', null),
+     (null, '2021-09-24 03:18:32.4', null, '1974-10-04 17:21:03.989')
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@alltypesorc_string
+POSTHOOK: query: INSERT INTO TABLE alltypesorc_string values
+     (false, '2021-09-24 03:18:32.4', '1985-11-18 16:37:54.0', '2010-04-08 02:43:35.861742727'),
+     (true, null, '1985-11-18 16:37:54.0', null),
+     (null, '2021-09-24 03:18:32.4', null, '1974-10-04 17:21:03.989')
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@alltypesorc_string
+POSTHOOK: Lineage: alltypesorc_string.cboolean1 SCRIPT []
+POSTHOOK: Lineage: alltypesorc_string.ctimestamp1 SCRIPT []
+POSTHOOK: Lineage: alltypesorc_string.ctimestamp2 SCRIPT []
+POSTHOOK: Lineage: alltypesorc_string.stimestamp1 SCRIPT []
 PREHOOK: query: CREATE TABLE alltypesorc_wrong(stimestamp1 string) STORED AS ORC
 PREHOOK: type: CREATETABLE
 PREHOOK: Output: database:default
@@ -72,7 +206,15 @@ PREHOOK: query: EXPLAIN VECTORIZATION EXPRESSION  SELECT
   weekofyear(ctimestamp1),
   hour(ctimestamp1),
   minute(ctimestamp1),
-  second(ctimestamp1)
+  second(ctimestamp1),
+  cboolean1,
+  ctimestamp1,
+  ctimestamp2,
+  if (cboolean1, ctimestamp1, timestamp '1319-02-02 16:31:57.778'),
+  if (cboolean1, timestamp '2000-12-18 08:42:30.0005', ctimestamp1),
+  if (cboolean1, ctimestamp1, ctimestamp2),
+  if (cboolean1, ctimestamp1, null),
+  if (cboolean1, null, ctimestamp2)
 FROM alltypesorc_string
 ORDER BY c1
 PREHOOK: type: QUERY
@@ -85,7 +227,15 @@ POSTHOOK: query: EXPLAIN VECTORIZATION EXPRESSION  SELECT
   weekofyear(ctimestamp1),
   hour(ctimestamp1),
   minute(ctimestamp1),
-  second(ctimestamp1)
+  second(ctimestamp1),
+  cboolean1,
+  ctimestamp1,
+  ctimestamp2,
+  if (cboolean1, ctimestamp1, timestamp '1319-02-02 16:31:57.778'),
+  if (cboolean1, timestamp '2000-12-18 08:42:30.0005', ctimestamp1),
+  if (cboolean1, ctimestamp1, ctimestamp2),
+  if (cboolean1, ctimestamp1, null),
+  if (cboolean1, null, ctimestamp2)
 FROM alltypesorc_string
 ORDER BY c1
 POSTHOOK: type: QUERY
@@ -108,18 +258,18 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: alltypesorc_string
-                  Statistics: Num rows: 40 Data size: 84 Basic stats: COMPLETE Column stats: NONE
+                  Statistics: Num rows: 52 Data size: 3555 Basic stats: COMPLETE Column stats: NONE
                   TableScan Vectorization:
                       native: true
                   Select Operator
-                    expressions: to_unix_timestamp(ctimestamp1) (type: bigint), year(ctimestamp1) (type: int), month(ctimestamp1) (type: int), day(ctimestamp1) (type: int), dayofmonth(ctimestamp1) (type: int), weekofyear(ctimestamp1) (type: int), hour(ctimestamp1) (type: int), minute(ctimestamp1) (type: int), second(ctimestamp1) (type: int)
-                    outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
+                    expressions: to_unix_timestamp(ctimestamp1) (type: bigint), year(ctimestamp1) (type: int), month(ctimestamp1) (type: int), day(ctimestamp1) (type: int), dayofmonth(ctimestamp1) (type: int), weekofyear(ctimestamp1) (type: int), hour(ctimestamp1) (type: int), minute(ctimestamp1) (type: int), second(ctimestamp1) (type: int), cboolean1 (type: boolean), ctimestamp1 (type: timestamp), ctimestamp2 (type: timestamp), if(cboolean1, ctimestamp1, TIMESTAMP'1319-02-02 16:31:57.778') (type: timestamp), if(cboolean1, TIMESTAMP'2000-12-18 08:42:30.0005', ctimestamp1) (type: timestamp), if(cboolean1, ctimestamp1, ctimestamp2) (type: timestamp), if(cboolean1, ctimestamp1, null) (type: timestamp), if(cboolean1, null, ctimestamp2) (type: timestamp)
+                    outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16
                     Select Vectorization:
                         className: VectorSelectOperator
                         native: true
-                        projectedOutputColumnNums: [3, 4, 5, 6, 7, 8, 9, 10, 11]
-                        selectExpressions: VectorUDFUnixTimeStampTimestamp(col 0:timestamp) -> 3:bigint, VectorUDFYearTimestamp(col 0:timestamp, field YEAR) -> 4:int, VectorUDFMonthTimestamp(col 0:timestamp, field MONTH) -> 5:int, VectorUDFDayOfMonthTimestamp(col 0:timestamp, field DAY_OF_MONTH) -> 6:int, VectorUDFDayOfMonthTimestamp(col 0:timestamp, field DAY_OF_MONTH) -> 7:int, VectorUDFWeekOfYearTimestamp(col 0:timestamp, field WEEK_OF_YEAR) -> 8:int, VectorUDFHourTimestamp(col 0:timestamp, field HOUR_OF_DAY) -> 9:int, VectorUDFMinuteTimestamp(col 0:timestamp, field MINUTE) -> 10:int, VectorUDFSecondTimestamp(col 0:timestamp, field SECOND) -> 11:int
-                    Statistics: Num rows: 40 Data size: 84 Basic stats: COMPLETE Column stats: NONE
+                        projectedOutputColumnNums: [5, 6, 7, 8, 9, 10, 11, 12, 13, 0, 1, 3, 14, 15, 16, 17, 18]
+                        selectExpressions: VectorUDFUnixTimeStampTimestamp(col 1:timestamp) -> 5:bigint, VectorUDFYearTimestamp(col 1:timestamp, field YEAR) -> 6:int, VectorUDFMonthTimestamp(col 1:timestamp, field MONTH) -> 7:int, VectorUDFDayOfMonthTimestamp(col 1:timestamp, field DAY_OF_MONTH) -> 8:int, VectorUDFDayOfMonthTimestamp(col 1:timestamp, field DAY_OF_MONTH) -> 9:int, VectorUDFWeekOfYearTimestamp(col 1:timestamp, field WEEK_OF_YEAR) -> 10:int, VectorUDFHourTimestamp(col 1:timestamp, field HOUR_OF_DAY) -> 11:int, VectorUDFMinuteTimestamp(col 1:timestamp, field MINUTE) -> 12:int, VectorUDFSecondTimestamp(col 1:timestamp, field SECOND) -> 13:int, IfExprTimestampColumnScalar(col 0:boolean, col 1:timestamp, val 1319-02-02 16:31:57.778) -> 14:timestamp, IfExprTimestampScalarColumn(col 0:boolean, val 2000-12-18 08:42:30.0005, col 1:timestamp) -> 15:timestamp, IfExprTimestampColumnColumn(col 0:boolean, col 1:timestampcol 3:timestamp) -> 16:timestamp, IfExprColumnNull(col 0:boole
 an, col 1:timestamp, null)(children: col 0:boolean, col 1:timestamp) -> 17:timestamp, IfExprNullColumn(col 0:boolean, null, col 3)(children: col 0:boolean, col 3:timestamp) -> 18:timestamp
+                    Statistics: Num rows: 52 Data size: 3555 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
                       key expressions: _col0 (type: bigint)
                       sort order: +
@@ -127,8 +277,8 @@ STAGE PLANS:
                           className: VectorReduceSinkObjectHashOperator
                           native: true
                           nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true
-                      Statistics: Num rows: 40 Data size: 84 Basic stats: COMPLETE Column stats: NONE
-                      value expressions: _col1 (type: int), _col2 (type: int), _col3 (type: int), _col4 (type: int), _col5 (type: int), _col6 (type: int), _col7 (type: int), _col8 (type: int)
+                      Statistics: Num rows: 52 Data size: 3555 Basic stats: COMPLETE Column stats: NONE
+                      value expressions: _col1 (type: int), _col2 (type: int), _col3 (type: int), _col4 (type: int), _col5 (type: int), _col6 (type: int), _col7 (type: int), _col8 (type: int), _col9 (type: boolean), _col10 (type: timestamp), _col11 (type: timestamp), _col12 (type: timestamp), _col13 (type: timestamp), _col14 (type: timestamp), _col15 (type: timestamp), _col16 (type: timestamp)
             Execution mode: vectorized
             Map Vectorization:
                 enabled: true
@@ -149,19 +299,19 @@ STAGE PLANS:
                 vectorized: true
             Reduce Operator Tree:
               Select Operator
-                expressions: KEY.reducesinkkey0 (type: bigint), VALUE._col0 (type: int), VALUE._col1 (type: int), VALUE._col2 (type: int), VALUE._col3 (type: int), VALUE._col4 (type: int), VALUE._col5 (type: int), VALUE._col6 (type: int), VALUE._col7 (type: int)
-                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
+                expressions: KEY.reducesinkkey0 (type: bigint), VALUE._col0 (type: int), VALUE._col1 (type: int), VALUE._col2 (type: int), VALUE._col3 (type: int), VALUE._col4 (type: int), VALUE._col5 (type: int), VALUE._col6 (type: int), VALUE._col7 (type: int), VALUE._col8 (type: boolean), VALUE._col9 (type: timestamp), VALUE._col10 (type: timestamp), VALUE._col11 (type: timestamp), VALUE._col12 (type: timestamp), VALUE._col13 (type: timestamp), VALUE._col14 (type: timestamp), VALUE._col15 (type: timestamp)
+                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16
                 Select Vectorization:
                     className: VectorSelectOperator
                     native: true
-                    projectedOutputColumnNums: [0, 1, 2, 3, 4, 5, 6, 7, 8]
-                Statistics: Num rows: 40 Data size: 84 Basic stats: COMPLETE Column stats: NONE
+                    projectedOutputColumnNums: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16]
+                Statistics: Num rows: 52 Data size: 3555 Basic stats: COMPLETE Column stats: NONE
                 File Output Operator
                   compressed: false
                   File Sink Vectorization:
                       className: VectorFileSinkOperator
                       native: false
-                  Statistics: Num rows: 40 Data size: 84 Basic stats: COMPLETE Column stats: NONE
+                  Statistics: Num rows: 52 Data size: 3555 Basic stats: COMPLETE Column stats: NONE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -182,7 +332,15 @@ PREHOOK: query: SELECT
   weekofyear(ctimestamp1),
   hour(ctimestamp1),
   minute(ctimestamp1),
-  second(ctimestamp1)
+  second(ctimestamp1),
+  cboolean1,
+  ctimestamp1,
+  ctimestamp2,
+  if (cboolean1, ctimestamp1, timestamp '1319-02-02 16:31:57.778'),
+  if (cboolean1, timestamp '2000-12-18 08:42:30.0005', ctimestamp1),
+  if (cboolean1, ctimestamp1, ctimestamp2),
+  if (cboolean1, ctimestamp1, null),
+  if (cboolean1, null, ctimestamp2)
 FROM alltypesorc_string
 ORDER BY c1
 PREHOOK: type: QUERY
@@ -197,52 +355,72 @@ POSTHOOK: query: SELECT
   weekofyear(ctimestamp1),
   hour(ctimestamp1),
   minute(ctimestamp1),
-  second(ctimestamp1)
+  second(ctimestamp1),
+  cboolean1,
+  ctimestamp1,
+  ctimestamp2,
+  if (cboolean1, ctimestamp1, timestamp '1319-02-02 16:31:57.778'),
+  if (cboolean1, timestamp '2000-12-18 08:42:30.0005', ctimestamp1),
+  if (cboolean1, ctimestamp1, ctimestamp2),
+  if (cboolean1, ctimestamp1, null),
+  if (cboolean1, null, ctimestamp2)
 FROM alltypesorc_string
 ORDER BY c1
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@alltypesorc_string
 #### A masked pattern was here ####
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
+-45479000681	528	10	27	27	43	8	15	18	true	0528-10-27 08:15:18.941718273	NULL	0528-10-27 08:15:18.941718273	2000-12-18 08:42:30.0005	0528-10-27 08:15:18.941718273	0528-10-27 08:15:18.941718273	NULL
+1632478712	2021	9	24	24	38	3	18	32	NULL	2021-09-24 03:18:32.4	1974-10-04 17:21:03.989	1319-02-02 16:31:57.778	2021-09-24 03:18:32.4	1974-10-04 17:21:03.989	NULL	1974-10-04 17:21:03.989
+1632478712	2021	9	24	24	38	3	18	32	false	2021-09-24 03:18:32.4	1999-10-03 16:59:10.396903939	1319-02-02 16:31:57.778	2021-09-24 03:18:32.4	1999-10-03 16:59:10.396903939	NULL	1999-10-03 16:59:10.396903939
+1632478712	2021	9	24	24	38	3	18	32	false	2021-09-24 03:18:32.4	2010-04-08 02:43:35.861742727	1319-02-02 16:31:57.778	2021-09-24 03:18:32.4	2010-04-08 02:43:35.861742727	NULL	2010-04-08 02:43:35.861742727
+1632478712	2021	9	24	24	38	3	18	32	false	2021-09-24 03:18:32.4	NULL	1319-02-02 16:31:57.778	2021-09-24 03:18:32.4	NULL	NULL	NULL
+163809612024	7160	12	2	2	48	6	0	24	NULL	7160-12-02 06:00:24.81200852	1966-08-16 13:36:50.183	1319-02-02 16:31:57.778	7160-12-02 06:00:24.81200852	1966-08-16 13:36:50.183	NULL	1966-08-16 13:36:50.183
+163809612024	7160	12	2	2	48	6	0	24	NULL	7160-12-02 06:00:24.81200852	NULL	1319-02-02 16:31:57.778	7160-12-02 06:00:24.81200852	NULL	NULL	NULL
+490725011	1985	7	20	20	29	9	30	11	true	1985-07-20 09:30:11	1319-02-02 16:31:57.778	1985-07-20 09:30:11	2000-12-18 08:42:30.0005	1985-07-20 09:30:11	1985-07-20 09:30:11	NULL
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	1319-02-02 16:31:57.778	NULL	NULL	NULL	NULL
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	false	NULL	1969-12-31 15:59:47.183	1319-02-02 16:31:57.778	NULL	1969-12-31 15:59:47.183	NULL	1969-12-31 15:59:47.183
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	false	NULL	1969-12-31 15:59:52.843	1319-02-02 16:31:57.778	NULL	1969-12-31 15:59:52.843	NULL	1969-12-31 15:59:52.843
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	false	NULL	1969-12-31 15:59:53.087	1319-02-02 16:31:57.778	NULL	1969-12-31 15:59:53.087	NULL	1969-12-31 15:59:53.087
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	false	NULL	1969-12-31 15:59:53.55	1319-02-02 16:31:57.778	NULL	1969-12-31 15:59:53.55	NULL	1969-12-31 15:59:53.55
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	false	NULL	1969-12-31 15:59:54.042	1319-02-02 16:31:57.778	NULL	1969-12-31 15:59:54.042	NULL	1969-12-31 15:59:54.042
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	false	NULL	1969-12-31 15:59:54.686	1319-02-02 16:31:57.778	NULL	1969-12-31 15:59:54.686	NULL	1969-12-31 15:59:54.686
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	false	NULL	1969-12-31 15:59:58.459	1319-02-02 16:31:57.778	NULL	1969-12-31 15:59:58.459	NULL	1969-12-31 15:59:58.459
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	false	NULL	1969-12-31 16:00:00.889	1319-02-02 16:31:57.778	NULL	1969-12-31 16:00:00.889	NULL	1969-12-31 16:00:00.889
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	false	NULL	1969-12-31 16:00:01.258	1319-02-02 16:31:57.778	NULL	1969-12-31 16:00:01.258	NULL	1969-12-31 16:00:01.258
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	false	NULL	1969-12-31 16:00:05.698	1319-02-02 16:31:57.778	NULL	1969-12-31 16:00:05.698	NULL	1969-12-31 16:00:05.698
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	false	NULL	1969-12-31 16:00:08.602	1319-02-02 16:31:57.778	NULL	1969-12-31 16:00:08.602	NULL	1969-12-31 16:00:08.602
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	false	NULL	1969-12-31 16:00:14.214	1319-02-02 16:31:57.778	NULL	1969-12-31 16:00:14.214	NULL	1969-12-31 16:00:14.214
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	false	NULL	1969-12-31 16:00:15.466	1319-02-02 16:31:57.778	NULL	1969-12-31 16:00:15.466	NULL	1969-12-31 16:00:15.466
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	false	NULL	NULL	1319-02-02 16:31:57.778	NULL	NULL	NULL	NULL
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	true	NULL	1969-12-31 15:59:46.123	NULL	2000-12-18 08:42:30.0005	NULL	NULL	NULL
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	true	NULL	1969-12-31 15:59:49.989	NULL	2000-12-18 08:42:30.0005	NULL	NULL	NULL
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	true	NULL	1969-12-31 15:59:51.119	NULL	2000-12-18 08:42:30.0005	NULL	NULL	NULL
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	true	NULL	1969-12-31 15:59:52.961	NULL	2000-12-18 08:42:30.0005	NULL	NULL	NULL
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	true	NULL	1969-12-31 15:59:52.967	NULL	2000-12-18 08:42:30.0005	NULL	NULL	NULL
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	true	NULL	1969-12-31 15:59:53.593	NULL	2000-12-18 08:42:30.0005	NULL	NULL	NULL
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	true	NULL	1969-12-31 15:59:53.641	NULL	2000-12-18 08:42:30.0005	NULL	NULL	NULL
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	true	NULL	1969-12-31 15:59:55.407	NULL	2000-12-18 08:42:30.0005	NULL	NULL	NULL
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	true	NULL	1969-12-31 15:59:55.439	NULL	2000-12-18 08:42:30.0005	NULL	NULL	NULL
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	true	NULL	1969-12-31 15:59:56.031	NULL	2000-12-18 08:42:30.0005	NULL	NULL	NULL
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	true	NULL	1969-12-31 15:59:57.719	NULL	2000-12-18 08:42:30.0005	NULL	NULL	NULL
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	true	NULL	1969-12-31 15:59:58.636	NULL	2000-12-18 08:42:30.0005	NULL	NULL	NULL
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	true	NULL	1969-12-31 16:00:00.176	NULL	2000-12-18 08:42:30.0005	NULL	NULL	NULL
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	true	NULL	1969-12-31 16:00:00.423	NULL	2000-12-18 08:42:30.0005	NULL	NULL	NULL
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	true	NULL	1969-12-31 16:00:00.477	NULL	2000-12-18 08:42:30.0005	NULL	NULL	NULL
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	true	NULL	1969-12-31 16:00:00.93	NULL	2000-12-18 08:42:30.0005	NULL	NULL	NULL
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	true	NULL	1969-12-31 16:00:01.839	NULL	2000-12-18 08:42:30.0005	NULL	NULL	NULL
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	true	NULL	1969-12-31 16:00:02.13	NULL	2000-12-18 08:42:30.0005	NULL	NULL	NULL
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	true	NULL	1969-12-31 16:00:03.151	NULL	2000-12-18 08:42:30.0005	NULL	NULL	NULL
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	true	NULL	1969-12-31 16:00:03.756	NULL	2000-12-18 08:42:30.0005	NULL	NULL	NULL
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	true	NULL	1969-12-31 16:00:06.134	NULL	2000-12-18 08:42:30.0005	NULL	NULL	NULL
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	true	NULL	1969-12-31 16:00:07.209	NULL	2000-12-18 08:42:30.0005	NULL	NULL	NULL
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	true	NULL	1969-12-31 16:00:10.361	NULL	2000-12-18 08:42:30.0005	NULL	NULL	NULL
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	true	NULL	1969-12-31 16:00:11.525	NULL	2000-12-18 08:42:30.0005	NULL	NULL	NULL
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	true	NULL	1969-12-31 16:00:13.589	NULL	2000-12-18 08:42:30.0005	NULL	NULL	NULL
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	true	NULL	1969-12-31 16:00:13.839	NULL	2000-12-18 08:42:30.0005	NULL	NULL	NULL
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	true	NULL	1969-12-31 16:00:15.601	NULL	2000-12-18 08:42:30.0005	NULL	NULL	NULL
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	true	NULL	2024-11-11 16:42:41.101	NULL	2000-12-18 08:42:30.0005	NULL	NULL	NULL
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	true	NULL	NULL	NULL	2000-12-18 08:42:30.0005	NULL	NULL	NULL
 PREHOOK: query: EXPLAIN VECTORIZATION EXPRESSION  SELECT
   to_unix_timestamp(stimestamp1) AS c1,
   year(stimestamp1),
@@ -288,7 +466,7 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: alltypesorc_string
-                  Statistics: Num rows: 40 Data size: 84 Basic stats: COMPLETE Column stats: NONE
+                  Statistics: Num rows: 52 Data size: 3555 Basic stats: COMPLETE Column stats: NONE
                   TableScan Vectorization:
                       native: true
                   Select Operator
@@ -297,9 +475,9 @@ STAGE PLANS:
                     Select Vectorization:
                         className: VectorSelectOperator
                         native: true
-                        projectedOutputColumnNums: [3, 4, 5, 6, 7, 8, 9, 10, 11]
-                        selectExpressions: VectorUDFUnixTimeStampString(col 1:string) -> 3:bigint, VectorUDFYearString(col 1:string, fieldStart 0, fieldLength 4) -> 4:int, VectorUDFMonthString(col 1:string, fieldStart 5, fieldLength 2) -> 5:int, VectorUDFDayOfMonthString(col 1:string, fieldStart 8, fieldLength 2) -> 6:int, VectorUDFDayOfMonthString(col 1:string, fieldStart 8, fieldLength 2) -> 7:int, VectorUDFWeekOfYearString(col 1:string) -> 8:int, VectorUDFHourString(col 1:string, fieldStart 11, fieldLength 2) -> 9:int, VectorUDFMinuteString(col 1:string, fieldStart 14, fieldLength 2) -> 10:int, VectorUDFSecondString(col 1:string, fieldStart 17, fieldLength 2) -> 11:int
-                    Statistics: Num rows: 40 Data size: 84 Basic stats: COMPLETE Column stats: NONE
+                        projectedOutputColumnNums: [5, 6, 7, 8, 9, 10, 11, 12, 13]
+                        selectExpressions: VectorUDFUnixTimeStampString(col 2:string) -> 5:bigint, VectorUDFYearString(col 2:string, fieldStart 0, fieldLength 4) -> 6:int, VectorUDFMonthString(col 2:string, fieldStart 5, fieldLength 2) -> 7:int, VectorUDFDayOfMonthString(col 2:string, fieldStart 8, fieldLength 2) -> 8:int, VectorUDFDayOfMonthString(col 2:string, fieldStart 8, fieldLength 2) -> 9:int, VectorUDFWeekOfYearString(col 2:string) -> 10:int, VectorUDFHourString(col 2:string, fieldStart 11, fieldLength 2) -> 11:int, VectorUDFMinuteString(col 2:string, fieldStart 14, fieldLength 2) -> 12:int, VectorUDFSecondString(col 2:string, fieldStart 17, fieldLength 2) -> 13:int
+                    Statistics: Num rows: 52 Data size: 3555 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
                       key expressions: _col0 (type: bigint)
                       sort order: +
@@ -307,7 +485,7 @@ STAGE PLANS:
                           className: VectorReduceSinkObjectHashOperator
                           native: true
                           nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true
-                      Statistics: Num rows: 40 Data size: 84 Basic stats: COMPLETE Column stats: NONE
+                      Statistics: Num rows: 52 Data size: 3555 Basic stats: COMPLETE Column stats: NONE
                       value expressions: _col1 (type: int), _col2 (type: int), _col3 (type: int), _col4 (type: int), _col5 (type: int), _col6 (type: int), _col7 (type: int), _col8 (type: int)
             Execution mode: vectorized
             Map Vectorization:
@@ -335,13 +513,13 @@ STAGE PLANS:
                     className: VectorSelectOperator
                     native: true
                     projectedOutputColumnNums: [0, 1, 2, 3, 4, 5, 6, 7, 8]
-                Statistics: Num rows: 40 Data size: 84 Basic stats: COMPLETE Column stats: NONE
+                Statistics: Num rows: 52 Data size: 3555 Basic stats: COMPLETE Column stats: NONE
                 File Output Operator
                   compressed: false
                   File Sink Vectorization:
                       className: VectorFileSinkOperator
                       native: false
-                  Statistics: Num rows: 40 Data size: 84 Basic stats: COMPLETE Column stats: NONE
+                  Statistics: Num rows: 52 Data size: 3555 Basic stats: COMPLETE Column stats: NONE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -383,6 +561,18 @@ ORDER BY c1
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@alltypesorc_string
 #### A masked pattern was here ####
+-2736243926	1883	4	17	17	16	4	14	34
+-62018170411	4	9	22	22	39	18	26	29
+1365579826	2013	4	10	10	15	0	43	46
+206731024925	8521	1	16	16	3	20	42	5
+271201265	1978	8	5	5	31	14	41	5
+501208674	1985	11	18	18	47	16	37	54
+501208674	1985	11	18	18	47	16	37	54
+94573848655	4966	12	4	4	49	9	30	55
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
 NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
 NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
 NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
@@ -468,7 +658,7 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: alltypesorc_string
-                  Statistics: Num rows: 40 Data size: 84 Basic stats: COMPLETE Column stats: NONE
+                  Statistics: Num rows: 52 Data size: 3555 Basic stats: COMPLETE Column stats: NONE
                   TableScan Vectorization:
                       native: true
                   Select Operator
@@ -477,9 +667,9 @@ STAGE PLANS:
                     Select Vectorization:
                         className: VectorSelectOperator
                         native: true
-                        projectedOutputColumnNums: [5, 6, 7, 8, 9, 10, 11, 12, 13]
-                        selectExpressions: LongColEqualLongColumn(col 3:bigint, col 4:bigint)(children: VectorUDFUnixTimeStampTimestamp(col 0:timestamp) -> 3:bigint, VectorUDFUnixTimeStampString(col 1:string) -> 4:bigint) -> 5:boolean, LongColEqualLongColumn(col 3:int, col 4:int)(children: VectorUDFYearTimestamp(col 0:timestamp, field YEAR) -> 3:int, VectorUDFYearString(col 1:string, fieldStart 0, fieldLength 4) -> 4:int) -> 6:boolean, LongColEqualLongColumn(col 3:int, col 4:int)(children: VectorUDFMonthTimestamp(col 0:timestamp, field MONTH) -> 3:int, VectorUDFMonthString(col 1:string, fieldStart 5, fieldLength 2) -> 4:int) -> 7:boolean, LongColEqualLongColumn(col 3:int, col 4:int)(children: VectorUDFDayOfMonthTimestamp(col 0:timestamp, field DAY_OF_MONTH) -> 3:int, VectorUDFDayOfMonthString(col 1:string, fieldStart 8, fieldLength 2) -> 4:int) -> 8:boolean, LongColEqualLongColumn(col 3:int, col 4:int)(children: VectorUDFDayOfMonthTimestamp(col 0:timestamp, field DAY_OF_MONTH) -> 3:
 int, VectorUDFDayOfMonthString(col 1:string, fieldStart 8, fieldLength 2) -> 4:int) -> 9:boolean, LongColEqualLongColumn(col 3:int, col 4:int)(children: VectorUDFWeekOfYearTimestamp(col 0:timestamp, field WEEK_OF_YEAR) -> 3:int, VectorUDFWeekOfYearString(col 1:string) -> 4:int) -> 10:boolean, LongColEqualLongColumn(col 3:int, col 4:int)(children: VectorUDFHourTimestamp(col 0:timestamp, field HOUR_OF_DAY) -> 3:int, VectorUDFHourString(col 1:string, fieldStart 11, fieldLength 2) -> 4:int) -> 11:boolean, LongColEqualLongColumn(col 3:int, col 4:int)(children: VectorUDFMinuteTimestamp(col 0:timestamp, field MINUTE) -> 3:int, VectorUDFMinuteString(col 1:string, fieldStart 14, fieldLength 2) -> 4:int) -> 12:boolean, LongColEqualLongColumn(col 3:int, col 4:int)(children: VectorUDFSecondTimestamp(col 0:timestamp, field SECOND) -> 3:int, VectorUDFSecondString(col 1:string, fieldStart 17, fieldLength 2) -> 4:int) -> 13:boolean
-                    Statistics: Num rows: 40 Data size: 84 Basic stats: COMPLETE Column stats: NONE
+                        projectedOutputColumnNums: [7, 8, 9, 10, 11, 12, 13, 14, 15]
+                        selectExpressions: LongColEqualLongColumn(col 5:bigint, col 6:bigint)(children: VectorUDFUnixTimeStampTimestamp(col 1:timestamp) -> 5:bigint, VectorUDFUnixTimeStampString(col 2:string) -> 6:bigint) -> 7:boolean, LongColEqualLongColumn(col 5:int, col 6:int)(children: VectorUDFYearTimestamp(col 1:timestamp, field YEAR) -> 5:int, VectorUDFYearString(col 2:string, fieldStart 0, fieldLength 4) -> 6:int) -> 8:boolean, LongColEqualLongColumn(col 5:int, col 6:int)(children: VectorUDFMonthTimestamp(col 1:timestamp, field MONTH) -> 5:int, VectorUDFMonthString(col 2:string, fieldStart 5, fieldLength 2) -> 6:int) -> 9:boolean, LongColEqualLongColumn(col 5:int, col 6:int)(children: VectorUDFDayOfMonthTimestamp(col 1:timestamp, field DAY_OF_MONTH) -> 5:int, VectorUDFDayOfMonthString(col 2:string, fieldStart 8, fieldLength 2) -> 6:int) -> 10:boolean, LongColEqualLongColumn(col 5:int, col 6:int)(children: VectorUDFDayOfMonthTimestamp(col 1:timestamp, field DAY_OF_MONTH) -> 5
 :int, VectorUDFDayOfMonthString(col 2:string, fieldStart 8, fieldLength 2) -> 6:int) -> 11:boolean, LongColEqualLongColumn(col 5:int, col 6:int)(children: VectorUDFWeekOfYearTimestamp(col 1:timestamp, field WEEK_OF_YEAR) -> 5:int, VectorUDFWeekOfYearString(col 2:string) -> 6:int) -> 12:boolean, LongColEqualLongColumn(col 5:int, col 6:int)(children: VectorUDFHourTimestamp(col 1:timestamp, field HOUR_OF_DAY) -> 5:int, VectorUDFHourString(col 2:string, fieldStart 11, fieldLength 2) -> 6:int) -> 13:boolean, LongColEqualLongColumn(col 5:int, col 6:int)(children: VectorUDFMinuteTimestamp(col 1:timestamp, field MINUTE) -> 5:int, VectorUDFMinuteString(col 2:string, fieldStart 14, fieldLength 2) -> 6:int) -> 14:boolean, LongColEqualLongColumn(col 5:int, col 6:int)(children: VectorUDFSecondTimestamp(col 1:timestamp, field SECOND) -> 5:int, VectorUDFSecondString(col 2:string, fieldStart 17, fieldLength 2) -> 6:int) -> 15:boolean
+                    Statistics: Num rows: 52 Data size: 3555 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
                       key expressions: _col0 (type: boolean)
                       sort order: +
@@ -487,7 +677,7 @@ STAGE PLANS:
                           className: VectorReduceSinkObjectHashOperator
                           native: true
                           nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true
-                      Statistics: Num rows: 40 Data size: 84 Basic stats: COMPLETE Column stats: NONE
+                      Statistics: Num rows: 52 Data size: 3555 Basic stats: COMPLETE Column stats: NONE
                       value expressions: _col1 (type: boolean), _col2 (type: boolean), _col3 (type: boolean), _col4 (type: boolean), _col5 (type: boolean), _col6 (type: boolean), _col7 (type: boolean), _col8 (type: boolean)
             Execution mode: vectorized
             Map Vectorization:
@@ -515,13 +705,13 @@ STAGE PLANS:
                     className: VectorSelectOperator
                     native: true
                     projectedOutputColumnNums: [0, 1, 2, 3, 4, 5, 6, 7, 8]
-                Statistics: Num rows: 40 Data size: 84 Basic stats: COMPLETE Column stats: NONE
+                Statistics: Num rows: 52 Data size: 3555 Basic stats: COMPLETE Column stats: NONE
                 File Output Operator
                   compressed: false
                   File Sink Vectorization:
                       className: VectorFileSinkOperator
                       native: false
-                  Statistics: Num rows: 40 Data size: 84 Basic stats: COMPLETE Column stats: NONE
+                  Statistics: Num rows: 52 Data size: 3555 Basic stats: COMPLETE Column stats: NONE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -603,6 +793,18 @@ NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
 NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
 NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
 NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
+NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
+false	false	false	false	false	false	false	false	false
+false	false	false	false	false	false	false	false	false
+false	false	false	false	false	false	false	false	false
+false	false	false	false	false	false	false	false	false
 PREHOOK: query: EXPLAIN VECTORIZATION EXPRESSION  SELECT
   to_unix_timestamp(stimestamp1) AS c1,
   year(stimestamp1),
@@ -779,7 +981,7 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: alltypesorc_string
-                  Statistics: Num rows: 40 Data size: 84 Basic stats: COMPLETE Column stats: NONE
+                  Statistics: Num rows: 52 Data size: 3555 Basic stats: COMPLETE Column stats: NONE
                   TableScan Vectorization:
                       native: true
                   Select Operator
@@ -788,12 +990,12 @@ STAGE PLANS:
                     Select Vectorization:
                         className: VectorSelectOperator
                         native: true
-                        projectedOutputColumnNums: [0]
-                    Statistics: Num rows: 40 Data size: 84 Basic stats: COMPLETE Column stats: NONE
+                        projectedOutputColumnNums: [1]
+                    Statistics: Num rows: 52 Data size: 3555 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
                       aggregations: min(ctimestamp1), max(ctimestamp1), count(ctimestamp1), count()
                       Group By Vectorization:
-                          aggregators: VectorUDAFMinTimestamp(col 0:timestamp) -> timestamp, VectorUDAFMaxTimestamp(col 0:timestamp) -> timestamp, VectorUDAFCount(col 0:timestamp) -> bigint, VectorUDAFCountStar(*) -> bigint
+                          aggregators: VectorUDAFMinTimestamp(col 1:timestamp) -> timestamp, VectorUDAFMaxTimestamp(col 1:timestamp) -> timestamp, VectorUDAFCount(col 1:timestamp) -> bigint, VectorUDAFCountStar(*) -> bigint
                           className: VectorGroupByOperator
                           groupByMode: HASH
                           native: false
@@ -876,7 +1078,7 @@ FROM alltypesorc_string
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@alltypesorc_string
 #### A masked pattern was here ####
-NULL	NULL	0	40
+0528-10-27 08:15:18.941718273	7160-12-02 06:00:24.81200852	8	52
 PREHOOK: query: EXPLAIN VECTORIZATION EXPRESSION  SELECT
   round(sum(ctimestamp1), 3)
 FROM alltypesorc_string
@@ -904,7 +1106,7 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: alltypesorc_string
-                  Statistics: Num rows: 40 Data size: 84 Basic stats: COMPLETE Column stats: NONE
+                  Statistics: Num rows: 52 Data size: 3555 Basic stats: COMPLETE Column stats: NONE
                   TableScan Vectorization:
                       native: true
                   Select Operator
@@ -913,12 +1115,12 @@ STAGE PLANS:
                     Select Vectorization:
                         className: VectorSelectOperator
                         native: true
-                        projectedOutputColumnNums: [0]
-                    Statistics: Num rows: 40 Data size: 84 Basic stats: COMPLETE Column stats: NONE
+                        projectedOutputColumnNums: [1]
+                    Statistics: Num rows: 52 Data size: 3555 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
                       aggregations: sum(ctimestamp1)
                       Group By Vectorization:
-                          aggregators: VectorUDAFSumTimestamp(col 0:timestamp) -> double
+                          aggregators: VectorUDAFSumTimestamp(col 1:timestamp) -> double
                           className: VectorGroupByOperator
                           groupByMode: HASH
                           native: false
@@ -1004,7 +1206,7 @@ FROM alltypesorc_string
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@alltypesorc_string
 #### A masked pattern was here ####
-NULL
+2.89160863229166E11
 PREHOOK: query: EXPLAIN VECTORIZATION EXPRESSION  SELECT
   round(avg(ctimestamp1), 0),
   variance(ctimestamp1) between 8.97077295279421E19 and 8.97077295279422E19,
@@ -1046,7 +1248,7 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: alltypesorc_string
-                  Statistics: Num rows: 40 Data size: 84 Basic stats: COMPLETE Column stats: NONE
+                  Statistics: Num rows: 52 Data size: 3555 Basic stats: COMPLETE Column stats: NONE
                   TableScan Vectorization:
                       native: true
                   Select Operator
@@ -1055,13 +1257,13 @@ STAGE PLANS:
                     Select Vectorization:
                         className: VectorSelectOperator
                         native: true
-                        projectedOutputColumnNums: [0, 3, 6]
-                        selectExpressions: CastTimestampToDouble(col 0:timestamp) -> 3:double, DoubleColMultiplyDoubleColumn(col 4:double, col 5:double)(children: CastTimestampToDouble(col 0:timestamp) -> 4:double, CastTimestampToDouble(col 0:timestamp) -> 5:double) -> 6:double
-                    Statistics: Num rows: 40 Data size: 84 Basic stats: COMPLETE Column stats: NONE
+                        projectedOutputColumnNums: [1, 5, 8]
+                        selectExpressions: CastTimestampToDouble(col 1:timestamp) -> 5:double, DoubleColMultiplyDoubleColumn(col 6:double, col 7:double)(children: CastTimestampToDouble(col 1:timestamp) -> 6:double, CastTimestampToDouble(col 1:timestamp) -> 7:double) -> 8:double
+                    Statistics: Num rows: 52 Data size: 3555 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
                       aggregations: sum(_col0), count(_col0), sum(_col2), sum(_col1)
                       Group By Vectorization:
-                          aggregators: VectorUDAFSumTimestamp(col 0:timestamp) -> double, VectorUDAFCount(col 0:timestamp) -> bigint, VectorUDAFSumDouble(col 6:double) -> double, VectorUDAFSumDouble(col 3:double) -> double
+                          aggregators: VectorUDAFSumTimestamp(col 1:timestamp) -> double, VectorUDAFCount(col 1:timestamp) -> bigint, VectorUDAFSumDouble(col 8:double) -> double, VectorUDAFSumDouble(col 5:double) -> double
                           className: VectorGroupByOperator
                           groupByMode: HASH
                           native: false
@@ -1161,4 +1363,4 @@ FROM alltypesorc_string
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@alltypesorc_string
 #### A masked pattern was here ####
-NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL
+3.6145107904E10	false	false	false	7.5245155692476E10	7.5245155692476E10	7.5245155692476E10	8.0440455033059E10


[35/50] [abbrv] hive git commit: HIVE-19433: HiveJoinPushTransitivePredicatesRule hangs (Vineet Garg, reviewed by Jesus Camacho Rodriguez)

Posted by vg...@apache.org.
HIVE-19433: HiveJoinPushTransitivePredicatesRule hangs (Vineet Garg,reviewed by Jesus Camacho Rodriguez)


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

Branch: refs/heads/branch-3.0.0
Commit: 8c12a11b762f5954f75eb35c10cb740dab632554
Parents: 71d211d
Author: Vineet Garg <vg...@apache.org>
Authored: Mon May 14 11:37:05 2018 -0700
Committer: Vineet Garg <vg...@apache.org>
Committed: Mon May 14 11:40:36 2018 -0700

----------------------------------------------------------------------
 .../calcite/stats/HiveRelMdPredicates.java      |   6 +-
 .../queries/clientpositive/infer_join_preds.q   | 222 +++++++
 .../clientpositive/infer_join_preds.q.out       | 594 +++++++++++++++++++
 3 files changed, 820 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/8c12a11b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/stats/HiveRelMdPredicates.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/stats/HiveRelMdPredicates.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/stats/HiveRelMdPredicates.java
index 0b1fe74..a137bdf 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/stats/HiveRelMdPredicates.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/stats/HiveRelMdPredicates.java
@@ -642,7 +642,7 @@ public class HiveRelMdPredicates implements MetadataHandler<BuiltInMetadata.Pred
         } else {
           computeNextMapping(iterationIdx.length - 1);
         }
-        return nextMapping != null;
+          return nextMapping != null;
       }
 
       public Mapping next() {
@@ -659,7 +659,9 @@ public class HiveRelMdPredicates implements MetadataHandler<BuiltInMetadata.Pred
           if (level == 0) {
             nextMapping = null;
           } else {
-            iterationIdx[level] = 0;
+            int tmp = columnSets[level].nextSetBit(0);
+            nextMapping.set(columns[level], tmp);
+            iterationIdx[level] = tmp + 1;
             computeNextMapping(level - 1);
           }
         } else {

http://git-wip-us.apache.org/repos/asf/hive/blob/8c12a11b/ql/src/test/queries/clientpositive/infer_join_preds.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/infer_join_preds.q b/ql/src/test/queries/clientpositive/infer_join_preds.q
index c2e0d09..a9f5468 100644
--- a/ql/src/test/queries/clientpositive/infer_join_preds.q
+++ b/ql/src/test/queries/clientpositive/infer_join_preds.q
@@ -59,3 +59,225 @@ select * from
 (select * from src)a
 right outer join
 (select * from src1 where 1 = 0)b on a.key = b.key;
+
+explain select * from src join src1 on src.key = src1.key and src.value = src1.value
+    where 4 between src.key and src.value;
+
+    CREATE TABLE `table1`(
+       `idp_warehouse_id` bigint,
+       `idp_audit_id` bigint,
+       `idp_effective_date` date,
+       `idp_end_date` date,
+       `idp_delete_date` date,
+       `pruid` varchar(32),
+       `prid` bigint,
+       `prtimesheetid` bigint,
+       `prassignmentid` bigint,
+       `prchargecodeid` bigint,
+       `prtypecodeid` bigint,
+       `prsequence` bigint,
+       `prmodby` varchar(96),
+       `prmodtime` timestamp,
+       `prrmexported` bigint,
+       `prrmckdel` bigint,
+       `slice_status` int,
+       `role_id` bigint,
+       `user_lov1` varchar(30),
+       `user_lov2` varchar(30),
+       `incident_id` bigint,
+       `incident_investment_id` bigint,
+       `odf_ss_actuals` bigint,
+       `practsum` decimal(38,20));
+
+    CREATE TABLE `table2`(
+       `idp_warehouse_id` bigint,
+       `idp_audit_id` bigint,
+       `idp_effective_date` date,
+       `idp_end_date` date,
+       `idp_delete_date` date,
+       `pruid` varchar(32),
+       `prid` bigint,
+       `prtimesheetid` bigint,
+       `prassignmentid` bigint,
+       `prchargecodeid` bigint,
+       `prtypecodeid` bigint,
+       `prsequence` bigint,
+       `prmodby` varchar(96),
+       `prmodtime` timestamp,
+       `prrmexported` bigint,
+       `prrmckdel` bigint,
+       `slice_status` int,
+       `role_id` bigint,
+       `user_lov1` varchar(30),
+       `user_lov2` varchar(30),
+       `incident_id` bigint,
+       `incident_investment_id` bigint,
+       `odf_ss_actuals` bigint,
+       `practsum` decimal(38,20));
+
+    explain SELECT          s.idp_warehouse_id AS source_warehouse_id
+    FROM            table1 s
+    JOIN
+
+                           table2 d
+    ON              (
+                                    s.prid = d.prid )
+    JOIN
+                             table2 e
+    ON
+                                    s.prid = e.prid
+    WHERE
+    concat(
+                    CASE
+                                    WHEN s.prid IS NULL THEN 1
+                                    ELSE s.prid
+                    END,',',
+                    CASE
+                                    WHEN s.prtimesheetid IS NULL THEN 1
+                                    ELSE s.prtimesheetid
+                    END,',',
+                    CASE
+                                    WHEN s.prassignmentid IS NULL THEN 1
+                                    ELSE s.prassignmentid
+                    END,',',
+                    CASE
+                                    WHEN s.prchargecodeid IS NULL THEN 1
+                                    ELSE s.prchargecodeid
+                    END,',',
+                    CASE
+                                    WHEN (s.prtypecodeid) IS NULL THEN ''
+                                    ELSE s.prtypecodeid
+                    END,',',
+                    CASE
+                                    WHEN s.practsum IS NULL THEN 1
+                                    ELSE s.practsum
+                    END,',',
+                    CASE
+                                    WHEN s.prsequence IS NULL THEN 1
+                                    ELSE s.prsequence
+                    END,',',
+                    CASE
+                                    WHEN length(s.prmodby) IS NULL THEN ''
+                                    ELSE s.prmodby
+                    END,',',
+                    CASE
+                                    WHEN s.prmodtime IS NULL THEN cast(from_unixtime(unix_timestamp('2017-12-08','yyyy-MM-dd') ) AS timestamp)
+                                    ELSE s.prmodtime
+                    END,',',
+                    CASE
+                                    WHEN s.prrmexported IS NULL THEN 1
+                                    ELSE s.prrmexported
+                    END,',',
+                    CASE
+                                    WHEN s.prrmckdel IS NULL THEN 1
+                                    ELSE s.prrmckdel
+                    END,',',
+                    CASE
+                                    WHEN s.slice_status IS NULL THEN 1
+                                    ELSE s.slice_status
+                    END,',',
+                    CASE
+                                    WHEN s.role_id IS NULL THEN 1
+                                    ELSE s.role_id
+                    END,',',
+                    CASE
+                                    WHEN length(s.user_lov1) IS NULL THEN ''
+                                    ELSE s.user_lov1
+                    END,',',
+                    CASE
+                                    WHEN length(s.user_lov2) IS NULL THEN ''
+                                    ELSE s.user_lov2
+                    END,',',
+                    CASE
+                                    WHEN s.incident_id IS NULL THEN 1
+                                    ELSE s.incident_id
+                    END,',',
+                    CASE
+                                    WHEN s.incident_investment_id IS NULL THEN 1
+                                    ELSE s.incident_investment_id
+                    END,',',
+                    CASE
+                                    WHEN s.odf_ss_actuals IS NULL THEN 1
+                                    ELSE s.odf_ss_actuals
+                    END ) != concat(
+                    CASE
+                                    WHEN length(d.pruid) IS NULL THEN ''
+                                    ELSE d.pruid
+                    END,',',
+                    CASE
+                                    WHEN d.prid IS NULL THEN 1
+                                    ELSE d.prid
+                    END,',',
+                    CASE
+                                    WHEN d.prtimesheetid IS NULL THEN 1
+                                    ELSE d.prtimesheetid
+                    END,',',
+                    CASE
+                                    WHEN d.prassignmentid IS NULL THEN 1
+                                    ELSE d.prassignmentid
+                    END,',',
+                    CASE
+                                    WHEN d.prchargecodeid IS NULL THEN 1
+                                    ELSE d.prchargecodeid
+                    END,',',
+                    CASE
+                                    WHEN (d.prtypecodeid) IS NULL THEN ''
+                                    ELSE d.prtypecodeid
+                    END,',',
+                    CASE
+                                    WHEN d.practsum IS NULL THEN 1
+                                    ELSE d.practsum
+                    END,',',
+                    CASE
+                                    WHEN d.prsequence IS NULL THEN 1
+                                    ELSE d.prsequence
+                    END,',',
+                    CASE
+                                    WHEN length(d.prmodby) IS NULL THEN ''
+                                    ELSE d.prmodby
+                    END,',',
+                    CASE
+                                    WHEN d.prmodtime IS NULL THEN cast(from_unixtime(unix_timestamp('2017-12-08','yyyy-MM-dd') ) AS timestamp)
+                                    ELSE d.prmodtime
+                    END,',',
+                    CASE
+                                    WHEN d.prrmexported IS NULL THEN 1
+                                    ELSE d.prrmexported
+                    END,',',
+                    CASE
+                                    WHEN d.prrmckdel IS NULL THEN 1
+                                    ELSE d.prrmckdel
+                    END,',',
+                    CASE
+                                    WHEN d.slice_status IS NULL THEN 1
+                                    ELSE d.slice_status
+                    END,',',
+                    CASE
+                                    WHEN d.role_id IS NULL THEN 1
+                                    ELSE d.role_id
+                    END,',',
+                    CASE
+                                    WHEN length(d.user_lov1) IS NULL THEN ''
+                                    ELSE d.user_lov1
+                    END,',',
+                    CASE
+                                    WHEN length(d.user_lov2) IS NULL THEN ''
+                                    ELSE d.user_lov2
+                    END,',',
+                    CASE
+                                    WHEN d.incident_id IS NULL THEN 1
+                                    ELSE d.incident_id
+                    END,',',
+                    CASE
+                                    WHEN d.incident_investment_id IS NULL THEN 1
+                                    ELSE d.incident_investment_id
+                    END,',',
+                    CASE
+                                    WHEN d.odf_ss_actuals IS NULL THEN 1
+                                    ELSE d.odf_ss_actuals
+                    END );
+
+drop table table2;
+drop table table1;
+
+

http://git-wip-us.apache.org/repos/asf/hive/blob/8c12a11b/ql/src/test/results/clientpositive/infer_join_preds.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/infer_join_preds.q.out b/ql/src/test/results/clientpositive/infer_join_preds.q.out
index 6a4fa34..a35faf3 100644
--- a/ql/src/test/results/clientpositive/infer_join_preds.q.out
+++ b/ql/src/test/results/clientpositive/infer_join_preds.q.out
@@ -607,3 +607,597 @@ POSTHOOK: type: QUERY
 POSTHOOK: Input: default@src
 POSTHOOK: Input: default@src1
 #### A masked pattern was here ####
+PREHOOK: query: explain select * from src join src1 on src.key = src1.key and src.value = src1.value
+    where 4 between src.key and src.value
+PREHOOK: type: QUERY
+POSTHOOK: query: explain select * from src join src1 on src.key = src1.key and src.value = src1.value
+    where 4 between src.key and src.value
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: src
+            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+            Filter Operator
+              predicate: (4.0D BETWEEN UDFToDouble(key) AND UDFToDouble(value) and key is not null and value is not null) (type: boolean)
+              Statistics: Num rows: 55 Data size: 584 Basic stats: COMPLETE Column stats: NONE
+              Select Operator
+                expressions: key (type: string), value (type: string)
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 55 Data size: 584 Basic stats: COMPLETE Column stats: NONE
+                Reduce Output Operator
+                  key expressions: _col0 (type: string), _col1 (type: string)
+                  sort order: ++
+                  Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
+                  Statistics: Num rows: 55 Data size: 584 Basic stats: COMPLETE Column stats: NONE
+          TableScan
+            alias: src1
+            Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
+            Filter Operator
+              predicate: (4.0D BETWEEN UDFToDouble(key) AND UDFToDouble(value) and key is not null and value is not null) (type: boolean)
+              Statistics: Num rows: 2 Data size: 15 Basic stats: COMPLETE Column stats: NONE
+              Select Operator
+                expressions: key (type: string), value (type: string)
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 2 Data size: 15 Basic stats: COMPLETE Column stats: NONE
+                Reduce Output Operator
+                  key expressions: _col0 (type: string), _col1 (type: string)
+                  sort order: ++
+                  Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
+                  Statistics: Num rows: 2 Data size: 15 Basic stats: COMPLETE Column stats: NONE
+      Reduce Operator Tree:
+        Join Operator
+          condition map:
+               Inner Join 0 to 1
+          keys:
+            0 _col0 (type: string), _col1 (type: string)
+            1 _col0 (type: string), _col1 (type: string)
+          outputColumnNames: _col0, _col1, _col2, _col3
+          Statistics: Num rows: 60 Data size: 642 Basic stats: COMPLETE Column stats: NONE
+          File Output Operator
+            compressed: false
+            Statistics: Num rows: 60 Data size: 642 Basic stats: COMPLETE Column stats: NONE
+            table:
+                input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: CREATE TABLE `table1`(
+       `idp_warehouse_id` bigint,
+       `idp_audit_id` bigint,
+       `idp_effective_date` date,
+       `idp_end_date` date,
+       `idp_delete_date` date,
+       `pruid` varchar(32),
+       `prid` bigint,
+       `prtimesheetid` bigint,
+       `prassignmentid` bigint,
+       `prchargecodeid` bigint,
+       `prtypecodeid` bigint,
+       `prsequence` bigint,
+       `prmodby` varchar(96),
+       `prmodtime` timestamp,
+       `prrmexported` bigint,
+       `prrmckdel` bigint,
+       `slice_status` int,
+       `role_id` bigint,
+       `user_lov1` varchar(30),
+       `user_lov2` varchar(30),
+       `incident_id` bigint,
+       `incident_investment_id` bigint,
+       `odf_ss_actuals` bigint,
+       `practsum` decimal(38,20))
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@table1
+POSTHOOK: query: CREATE TABLE `table1`(
+       `idp_warehouse_id` bigint,
+       `idp_audit_id` bigint,
+       `idp_effective_date` date,
+       `idp_end_date` date,
+       `idp_delete_date` date,
+       `pruid` varchar(32),
+       `prid` bigint,
+       `prtimesheetid` bigint,
+       `prassignmentid` bigint,
+       `prchargecodeid` bigint,
+       `prtypecodeid` bigint,
+       `prsequence` bigint,
+       `prmodby` varchar(96),
+       `prmodtime` timestamp,
+       `prrmexported` bigint,
+       `prrmckdel` bigint,
+       `slice_status` int,
+       `role_id` bigint,
+       `user_lov1` varchar(30),
+       `user_lov2` varchar(30),
+       `incident_id` bigint,
+       `incident_investment_id` bigint,
+       `odf_ss_actuals` bigint,
+       `practsum` decimal(38,20))
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@table1
+PREHOOK: query: CREATE TABLE `table2`(
+       `idp_warehouse_id` bigint,
+       `idp_audit_id` bigint,
+       `idp_effective_date` date,
+       `idp_end_date` date,
+       `idp_delete_date` date,
+       `pruid` varchar(32),
+       `prid` bigint,
+       `prtimesheetid` bigint,
+       `prassignmentid` bigint,
+       `prchargecodeid` bigint,
+       `prtypecodeid` bigint,
+       `prsequence` bigint,
+       `prmodby` varchar(96),
+       `prmodtime` timestamp,
+       `prrmexported` bigint,
+       `prrmckdel` bigint,
+       `slice_status` int,
+       `role_id` bigint,
+       `user_lov1` varchar(30),
+       `user_lov2` varchar(30),
+       `incident_id` bigint,
+       `incident_investment_id` bigint,
+       `odf_ss_actuals` bigint,
+       `practsum` decimal(38,20))
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@table2
+POSTHOOK: query: CREATE TABLE `table2`(
+       `idp_warehouse_id` bigint,
+       `idp_audit_id` bigint,
+       `idp_effective_date` date,
+       `idp_end_date` date,
+       `idp_delete_date` date,
+       `pruid` varchar(32),
+       `prid` bigint,
+       `prtimesheetid` bigint,
+       `prassignmentid` bigint,
+       `prchargecodeid` bigint,
+       `prtypecodeid` bigint,
+       `prsequence` bigint,
+       `prmodby` varchar(96),
+       `prmodtime` timestamp,
+       `prrmexported` bigint,
+       `prrmckdel` bigint,
+       `slice_status` int,
+       `role_id` bigint,
+       `user_lov1` varchar(30),
+       `user_lov2` varchar(30),
+       `incident_id` bigint,
+       `incident_investment_id` bigint,
+       `odf_ss_actuals` bigint,
+       `practsum` decimal(38,20))
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@table2
+PREHOOK: query: explain SELECT          s.idp_warehouse_id AS source_warehouse_id
+    FROM            table1 s
+    JOIN
+
+                           table2 d
+    ON              (
+                                    s.prid = d.prid )
+    JOIN
+                             table2 e
+    ON
+                                    s.prid = e.prid
+    WHERE
+    concat(
+                    CASE
+                                    WHEN s.prid IS NULL THEN 1
+                                    ELSE s.prid
+                    END,',',
+                    CASE
+                                    WHEN s.prtimesheetid IS NULL THEN 1
+                                    ELSE s.prtimesheetid
+                    END,',',
+                    CASE
+                                    WHEN s.prassignmentid IS NULL THEN 1
+                                    ELSE s.prassignmentid
+                    END,',',
+                    CASE
+                                    WHEN s.prchargecodeid IS NULL THEN 1
+                                    ELSE s.prchargecodeid
+                    END,',',
+                    CASE
+                                    WHEN (s.prtypecodeid) IS NULL THEN ''
+                                    ELSE s.prtypecodeid
+                    END,',',
+                    CASE
+                                    WHEN s.practsum IS NULL THEN 1
+                                    ELSE s.practsum
+                    END,',',
+                    CASE
+                                    WHEN s.prsequence IS NULL THEN 1
+                                    ELSE s.prsequence
+                    END,',',
+                    CASE
+                                    WHEN length(s.prmodby) IS NULL THEN ''
+                                    ELSE s.prmodby
+                    END,',',
+                    CASE
+                                    WHEN s.prmodtime IS NULL THEN cast(from_unixtime(unix_timestamp('2017-12-08','yyyy-MM-dd') ) AS timestamp)
+                                    ELSE s.prmodtime
+                    END,',',
+                    CASE
+                                    WHEN s.prrmexported IS NULL THEN 1
+                                    ELSE s.prrmexported
+                    END,',',
+                    CASE
+                                    WHEN s.prrmckdel IS NULL THEN 1
+                                    ELSE s.prrmckdel
+                    END,',',
+                    CASE
+                                    WHEN s.slice_status IS NULL THEN 1
+                                    ELSE s.slice_status
+                    END,',',
+                    CASE
+                                    WHEN s.role_id IS NULL THEN 1
+                                    ELSE s.role_id
+                    END,',',
+                    CASE
+                                    WHEN length(s.user_lov1) IS NULL THEN ''
+                                    ELSE s.user_lov1
+                    END,',',
+                    CASE
+                                    WHEN length(s.user_lov2) IS NULL THEN ''
+                                    ELSE s.user_lov2
+                    END,',',
+                    CASE
+                                    WHEN s.incident_id IS NULL THEN 1
+                                    ELSE s.incident_id
+                    END,',',
+                    CASE
+                                    WHEN s.incident_investment_id IS NULL THEN 1
+                                    ELSE s.incident_investment_id
+                    END,',',
+                    CASE
+                                    WHEN s.odf_ss_actuals IS NULL THEN 1
+                                    ELSE s.odf_ss_actuals
+                    END ) != concat(
+                    CASE
+                                    WHEN length(d.pruid) IS NULL THEN ''
+                                    ELSE d.pruid
+                    END,',',
+                    CASE
+                                    WHEN d.prid IS NULL THEN 1
+                                    ELSE d.prid
+                    END,',',
+                    CASE
+                                    WHEN d.prtimesheetid IS NULL THEN 1
+                                    ELSE d.prtimesheetid
+                    END,',',
+                    CASE
+                                    WHEN d.prassignmentid IS NULL THEN 1
+                                    ELSE d.prassignmentid
+                    END,',',
+                    CASE
+                                    WHEN d.prchargecodeid IS NULL THEN 1
+                                    ELSE d.prchargecodeid
+                    END,',',
+                    CASE
+                                    WHEN (d.prtypecodeid) IS NULL THEN ''
+                                    ELSE d.prtypecodeid
+                    END,',',
+                    CASE
+                                    WHEN d.practsum IS NULL THEN 1
+                                    ELSE d.practsum
+                    END,',',
+                    CASE
+                                    WHEN d.prsequence IS NULL THEN 1
+                                    ELSE d.prsequence
+                    END,',',
+                    CASE
+                                    WHEN length(d.prmodby) IS NULL THEN ''
+                                    ELSE d.prmodby
+                    END,',',
+                    CASE
+                                    WHEN d.prmodtime IS NULL THEN cast(from_unixtime(unix_timestamp('2017-12-08','yyyy-MM-dd') ) AS timestamp)
+                                    ELSE d.prmodtime
+                    END,',',
+                    CASE
+                                    WHEN d.prrmexported IS NULL THEN 1
+                                    ELSE d.prrmexported
+                    END,',',
+                    CASE
+                                    WHEN d.prrmckdel IS NULL THEN 1
+                                    ELSE d.prrmckdel
+                    END,',',
+                    CASE
+                                    WHEN d.slice_status IS NULL THEN 1
+                                    ELSE d.slice_status
+                    END,',',
+                    CASE
+                                    WHEN d.role_id IS NULL THEN 1
+                                    ELSE d.role_id
+                    END,',',
+                    CASE
+                                    WHEN length(d.user_lov1) IS NULL THEN ''
+                                    ELSE d.user_lov1
+                    END,',',
+                    CASE
+                                    WHEN length(d.user_lov2) IS NULL THEN ''
+                                    ELSE d.user_lov2
+                    END,',',
+                    CASE
+                                    WHEN d.incident_id IS NULL THEN 1
+                                    ELSE d.incident_id
+                    END,',',
+                    CASE
+                                    WHEN d.incident_investment_id IS NULL THEN 1
+                                    ELSE d.incident_investment_id
+                    END,',',
+                    CASE
+                                    WHEN d.odf_ss_actuals IS NULL THEN 1
+                                    ELSE d.odf_ss_actuals
+                    END )
+PREHOOK: type: QUERY
+POSTHOOK: query: explain SELECT          s.idp_warehouse_id AS source_warehouse_id
+    FROM            table1 s
+    JOIN
+
+                           table2 d
+    ON              (
+                                    s.prid = d.prid )
+    JOIN
+                             table2 e
+    ON
+                                    s.prid = e.prid
+    WHERE
+    concat(
+                    CASE
+                                    WHEN s.prid IS NULL THEN 1
+                                    ELSE s.prid
+                    END,',',
+                    CASE
+                                    WHEN s.prtimesheetid IS NULL THEN 1
+                                    ELSE s.prtimesheetid
+                    END,',',
+                    CASE
+                                    WHEN s.prassignmentid IS NULL THEN 1
+                                    ELSE s.prassignmentid
+                    END,',',
+                    CASE
+                                    WHEN s.prchargecodeid IS NULL THEN 1
+                                    ELSE s.prchargecodeid
+                    END,',',
+                    CASE
+                                    WHEN (s.prtypecodeid) IS NULL THEN ''
+                                    ELSE s.prtypecodeid
+                    END,',',
+                    CASE
+                                    WHEN s.practsum IS NULL THEN 1
+                                    ELSE s.practsum
+                    END,',',
+                    CASE
+                                    WHEN s.prsequence IS NULL THEN 1
+                                    ELSE s.prsequence
+                    END,',',
+                    CASE
+                                    WHEN length(s.prmodby) IS NULL THEN ''
+                                    ELSE s.prmodby
+                    END,',',
+                    CASE
+                                    WHEN s.prmodtime IS NULL THEN cast(from_unixtime(unix_timestamp('2017-12-08','yyyy-MM-dd') ) AS timestamp)
+                                    ELSE s.prmodtime
+                    END,',',
+                    CASE
+                                    WHEN s.prrmexported IS NULL THEN 1
+                                    ELSE s.prrmexported
+                    END,',',
+                    CASE
+                                    WHEN s.prrmckdel IS NULL THEN 1
+                                    ELSE s.prrmckdel
+                    END,',',
+                    CASE
+                                    WHEN s.slice_status IS NULL THEN 1
+                                    ELSE s.slice_status
+                    END,',',
+                    CASE
+                                    WHEN s.role_id IS NULL THEN 1
+                                    ELSE s.role_id
+                    END,',',
+                    CASE
+                                    WHEN length(s.user_lov1) IS NULL THEN ''
+                                    ELSE s.user_lov1
+                    END,',',
+                    CASE
+                                    WHEN length(s.user_lov2) IS NULL THEN ''
+                                    ELSE s.user_lov2
+                    END,',',
+                    CASE
+                                    WHEN s.incident_id IS NULL THEN 1
+                                    ELSE s.incident_id
+                    END,',',
+                    CASE
+                                    WHEN s.incident_investment_id IS NULL THEN 1
+                                    ELSE s.incident_investment_id
+                    END,',',
+                    CASE
+                                    WHEN s.odf_ss_actuals IS NULL THEN 1
+                                    ELSE s.odf_ss_actuals
+                    END ) != concat(
+                    CASE
+                                    WHEN length(d.pruid) IS NULL THEN ''
+                                    ELSE d.pruid
+                    END,',',
+                    CASE
+                                    WHEN d.prid IS NULL THEN 1
+                                    ELSE d.prid
+                    END,',',
+                    CASE
+                                    WHEN d.prtimesheetid IS NULL THEN 1
+                                    ELSE d.prtimesheetid
+                    END,',',
+                    CASE
+                                    WHEN d.prassignmentid IS NULL THEN 1
+                                    ELSE d.prassignmentid
+                    END,',',
+                    CASE
+                                    WHEN d.prchargecodeid IS NULL THEN 1
+                                    ELSE d.prchargecodeid
+                    END,',',
+                    CASE
+                                    WHEN (d.prtypecodeid) IS NULL THEN ''
+                                    ELSE d.prtypecodeid
+                    END,',',
+                    CASE
+                                    WHEN d.practsum IS NULL THEN 1
+                                    ELSE d.practsum
+                    END,',',
+                    CASE
+                                    WHEN d.prsequence IS NULL THEN 1
+                                    ELSE d.prsequence
+                    END,',',
+                    CASE
+                                    WHEN length(d.prmodby) IS NULL THEN ''
+                                    ELSE d.prmodby
+                    END,',',
+                    CASE
+                                    WHEN d.prmodtime IS NULL THEN cast(from_unixtime(unix_timestamp('2017-12-08','yyyy-MM-dd') ) AS timestamp)
+                                    ELSE d.prmodtime
+                    END,',',
+                    CASE
+                                    WHEN d.prrmexported IS NULL THEN 1
+                                    ELSE d.prrmexported
+                    END,',',
+                    CASE
+                                    WHEN d.prrmckdel IS NULL THEN 1
+                                    ELSE d.prrmckdel
+                    END,',',
+                    CASE
+                                    WHEN d.slice_status IS NULL THEN 1
+                                    ELSE d.slice_status
+                    END,',',
+                    CASE
+                                    WHEN d.role_id IS NULL THEN 1
+                                    ELSE d.role_id
+                    END,',',
+                    CASE
+                                    WHEN length(d.user_lov1) IS NULL THEN ''
+                                    ELSE d.user_lov1
+                    END,',',
+                    CASE
+                                    WHEN length(d.user_lov2) IS NULL THEN ''
+                                    ELSE d.user_lov2
+                    END,',',
+                    CASE
+                                    WHEN d.incident_id IS NULL THEN 1
+                                    ELSE d.incident_id
+                    END,',',
+                    CASE
+                                    WHEN d.incident_investment_id IS NULL THEN 1
+                                    ELSE d.incident_investment_id
+                    END,',',
+                    CASE
+                                    WHEN d.odf_ss_actuals IS NULL THEN 1
+                                    ELSE d.odf_ss_actuals
+                    END )
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: s
+            Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+            Filter Operator
+              predicate: prid is not null (type: boolean)
+              Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+              Reduce Output Operator
+                key expressions: prid (type: bigint)
+                sort order: +
+                Map-reduce partition columns: prid (type: bigint)
+                Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                value expressions: idp_warehouse_id (type: bigint), prtimesheetid (type: bigint), prassignmentid (type: bigint), prchargecodeid (type: bigint), prtypecodeid (type: bigint), prsequence (type: bigint), prmodby (type: varchar(96)), prmodtime (type: timestamp), prrmexported (type: bigint), prrmckdel (type: bigint), slice_status (type: int), role_id (type: bigint), user_lov1 (type: varchar(30)), user_lov2 (type: varchar(30)), incident_id (type: bigint), incident_investment_id (type: bigint), odf_ss_actuals (type: bigint), practsum (type: decimal(38,20))
+          TableScan
+            alias: d
+            Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+            Filter Operator
+              predicate: prid is not null (type: boolean)
+              Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+              Reduce Output Operator
+                key expressions: prid (type: bigint)
+                sort order: +
+                Map-reduce partition columns: prid (type: bigint)
+                Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                value expressions: pruid (type: varchar(32)), prtimesheetid (type: bigint), prassignmentid (type: bigint), prchargecodeid (type: bigint), prtypecodeid (type: bigint), prsequence (type: bigint), prmodby (type: varchar(96)), prmodtime (type: timestamp), prrmexported (type: bigint), prrmckdel (type: bigint), slice_status (type: int), role_id (type: bigint), user_lov1 (type: varchar(30)), user_lov2 (type: varchar(30)), incident_id (type: bigint), incident_investment_id (type: bigint), odf_ss_actuals (type: bigint), practsum (type: decimal(38,20))
+          TableScan
+            alias: e
+            Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+            Filter Operator
+              predicate: prid is not null (type: boolean)
+              Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+              Reduce Output Operator
+                key expressions: prid (type: bigint)
+                sort order: +
+                Map-reduce partition columns: prid (type: bigint)
+                Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+      Reduce Operator Tree:
+        Join Operator
+          condition map:
+               Inner Join 0 to 1
+               Inner Join 0 to 2
+          keys:
+            0 prid (type: bigint)
+            1 prid (type: bigint)
+            2 prid (type: bigint)
+          outputColumnNames: _col0, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col19, _col20, _col21, _col22, _col23, _col32, _col33, _col34, _col35, _col36, _col37, _col38, _col39, _col40, _col41, _col42, _col43, _col44, _col45, _col46, _col47, _col48, _col49, _col50
+          Statistics: Num rows: 2 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+          Filter Operator
+            predicate: (concat(CASE WHEN (_col6 is null) THEN (1) ELSE (_col6) END, ',', CASE WHEN (_col7 is null) THEN (1) ELSE (_col7) END, ',', CASE WHEN (_col8 is null) THEN (1) ELSE (_col8) END, ',', CASE WHEN (_col9 is null) THEN (1) ELSE (_col9) END, ',', CASE WHEN (_col10 is null) THEN ('') ELSE (_col10) END, ',', CASE WHEN (_col23 is null) THEN (1) ELSE (_col23) END, ',', CASE WHEN (_col11 is null) THEN (1) ELSE (_col11) END, ',', CASE WHEN (length(_col12) is null) THEN ('') ELSE (_col12) END, ',', CASE WHEN (_col13 is null) THEN (TIMESTAMP'2017-12-08 00:00:00.0') ELSE (_col13) END, ',', CASE WHEN (_col14 is null) THEN (1) ELSE (_col14) END, ',', CASE WHEN (_col15 is null) THEN (1) ELSE (_col15) END, ',', CASE WHEN (_col16 is null) THEN (1) ELSE (_col16) END, ',', CASE WHEN (_col17 is null) THEN (1) ELSE (_col17) END, ',', CASE WHEN (length(_col18) is null) THEN ('') ELSE (_col18) END, ',', CASE WHEN (length(_col19) is null) THEN ('') ELSE (_col19) END, ',', CASE WHEN (_col
 20 is null) THEN (1) ELSE (_col20) END, ',', CASE WHEN (_col21 is null) THEN (1) ELSE (_col21) END, ',', CASE WHEN (_col22 is null) THEN (1) ELSE (_col22) END) <> concat(CASE WHEN (length(_col32) is null) THEN ('') ELSE (_col32) END, ',', CASE WHEN (_col33 is null) THEN (1) ELSE (_col33) END, ',', CASE WHEN (_col34 is null) THEN (1) ELSE (_col34) END, ',', CASE WHEN (_col35 is null) THEN (1) ELSE (_col35) END, ',', CASE WHEN (_col36 is null) THEN (1) ELSE (_col36) END, ',', CASE WHEN (_col37 is null) THEN ('') ELSE (_col37) END, ',', CASE WHEN (_col50 is null) THEN (1) ELSE (_col50) END, ',', CASE WHEN (_col38 is null) THEN (1) ELSE (_col38) END, ',', CASE WHEN (length(_col39) is null) THEN ('') ELSE (_col39) END, ',', CASE WHEN (_col40 is null) THEN (TIMESTAMP'2017-12-08 00:00:00.0') ELSE (_col40) END, ',', CASE WHEN (_col41 is null) THEN (1) ELSE (_col41) END, ',', CASE WHEN (_col42 is null) THEN (1) ELSE (_col42) END, ',', CASE WHEN (_col43 is null) THEN (1) ELSE (_col43) END, ',
 ', CASE WHEN (_col44 is null) THEN (1) ELSE (_col44) END, ',', CASE WHEN (length(_col45) is null) THEN ('') ELSE (_col45) END, ',', CASE WHEN (length(_col46) is null) THEN ('') ELSE (_col46) END, ',', CASE WHEN (_col47 is null) THEN (1) ELSE (_col47) END, ',', CASE WHEN (_col48 is null) THEN (1) ELSE (_col48) END, ',', CASE WHEN (_col49 is null) THEN (1) ELSE (_col49) END)) (type: boolean)
+            Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+            Select Operator
+              expressions: _col0 (type: bigint)
+              outputColumnNames: _col0
+              Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+              File Output Operator
+                compressed: false
+                Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                table:
+                    input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                    output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                    serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: drop table table2
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: default@table2
+PREHOOK: Output: default@table2
+POSTHOOK: query: drop table table2
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: default@table2
+POSTHOOK: Output: default@table2
+PREHOOK: query: drop table table1
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: default@table1
+PREHOOK: Output: default@table1
+POSTHOOK: query: drop table table1
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: default@table1
+POSTHOOK: Output: default@table1