You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by ha...@apache.org on 2013/10/17 17:12:09 UTC

svn commit: r1533106 - in /hive/trunk/ql/src: java/org/apache/hadoop/hive/ql/exec/vector/ java/org/apache/hadoop/hive/ql/exec/vector/expressions/ java/org/apache/hadoop/hive/ql/optimizer/physical/ test/org/apache/hadoop/hive/ql/exec/vector/expressions/...

Author: hashutosh
Date: Thu Oct 17 15:12:09 2013
New Revision: 1533106

URL: http://svn.apache.org/r1533106
Log:
HIVE-5517 : Implement end-to-end tests for vectorized string and math functions, and casts (Eric Hanson via Ashutosh Chauhan)

Added:
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/MathFuncLongToLong.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/PosModDoubleToDouble.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/PosModLongToLong.java
    hive/trunk/ql/src/test/queries/clientpositive/vectorized_casts.q
    hive/trunk/ql/src/test/queries/clientpositive/vectorized_math_funcs.q
    hive/trunk/ql/src/test/queries/clientpositive/vectorized_string_funcs.q
    hive/trunk/ql/src/test/results/clientpositive/vectorized_casts.q.out
    hive/trunk/ql/src/test/results/clientpositive/vectorized_math_funcs.q.out
    hive/trunk/ql/src/test/results/clientpositive/vectorized_string_funcs.q.out
Modified:
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/Vectorizer.java
    hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorMathFunctions.java

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java?rev=1533106&r1=1533105&r2=1533106&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java Thu Oct 17 15:12:09 2013
@@ -115,6 +115,7 @@ import org.apache.hadoop.hive.ql.udf.UDF
 import org.apache.hadoop.hive.ql.udf.UDFOPNegative;
 import org.apache.hadoop.hive.ql.udf.UDFOPPlus;
 import org.apache.hadoop.hive.ql.udf.UDFOPPositive;
+import org.apache.hadoop.hive.ql.udf.UDFPosMod;
 import org.apache.hadoop.hive.ql.udf.UDFPower;
 import org.apache.hadoop.hive.ql.udf.UDFRegExp;
 import org.apache.hadoop.hive.ql.udf.UDFRTrim;
@@ -737,11 +738,45 @@ public class VectorizationContext {
       return getCastToDoubleExpression(childExpr);
     } else if (cl.equals(UDFToString.class)) {
       return getCastToString(childExpr);
+    } else if (cl.equals(UDFPosMod.class)) {
+      return getPosModExpression(childExpr);
     }
 
     throw new HiveException("Udf: "+udf.getClass().getSimpleName()+", is not supported");
   }
 
+  private VectorExpression getPosModExpression(List<ExprNodeDesc> childExpr)
+      throws HiveException {
+    String inputType = childExpr.get(0).getTypeString();
+
+    if (isIntFamily(inputType)) {
+      // Try to get the second argument (the modulo divisor)
+      long divisor = getLongScalar(childExpr.get(1));
+
+      // Use the standard logic for a unary function to handle the first argument.
+      VectorExpression e = getUnaryFunctionExpression("PosMod", "Long", childExpr,
+          CUSTOM_EXPR_PACKAGE);
+
+      // Set second argument for this special case
+      ((ISetLongArg) e).setArg(divisor);
+      return e;
+    } else if (isFloatFamily(inputType)) {
+
+      // Try to get the second argument (the modulo divisor)
+      double divisor = getDoubleScalar(childExpr.get(1));
+
+      // Use the standard logic for a unary function to handle the first argument.
+      VectorExpression e = getUnaryFunctionExpression("PosMod", "Double", childExpr,
+          CUSTOM_EXPR_PACKAGE);
+
+      // Set second argument for this special case
+      ((ISetDoubleArg) e).setArg(divisor);
+      return e;
+    }
+
+    throw new HiveException("Unhandled input type for PMOD():  " + inputType);
+  }
+
   private VectorExpression getCastToTimestamp(List<ExprNodeDesc> childExpr)
       throws HiveException {
     String inputType = childExpr.get(0).getTypeString();
@@ -905,16 +940,19 @@ public class VectorizationContext {
       return getUnaryFunctionExpression("FuncLn", "Double", childExpr,
           GENERATED_EXPR_PACKAGE);
     } else if (childExpr.size() == 2) {
-      String argType = childExpr.get(0).getTypeString();
 
-      // Try to get the second argument, typically a constant value (the base)
-      double base = getDoubleScalar(childExpr.get(1));
+      // Get the type of the (normally variable) input expression
+      String argType = childExpr.get(1).getTypeString();
 
-      // Use the standard logic for a unary function to handle the first argument.
-      VectorExpression e = getUnaryFunctionExpression("FuncLogWithBase", "Double", childExpr,
+      // Try to get the first argument, typically a constant value (the base)
+      double base = getDoubleScalar(childExpr.get(0));
+
+      // Use the standard logic for a unary function to handle the second argument.
+      VectorExpression e = getUnaryFunctionExpression("FuncLogWithBase", "Double",
+          childExpr.subList(1, 2), // pass the second argument as the first
           CUSTOM_EXPR_PACKAGE);
 
-      // set the second argument for this special case
+      // set the first argument (the base) for this special case
       ((ISetDoubleArg) e).setArg(base);
       return e;
     }

Added: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/MathFuncLongToLong.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/MathFuncLongToLong.java?rev=1533106&view=auto
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/MathFuncLongToLong.java (added)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/MathFuncLongToLong.java Thu Oct 17 15:12:09 2013
@@ -0,0 +1,128 @@
+/**
+ * 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.DoubleColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+
+/**
+ * Implement vectorized math function that takes a long (and optionally additional
+ * constant argument(s)) and returns long. Use this for PMOD() etc.
+ *
+ * Do NOT use this for simple math functions like sin/cos/exp etc. that just take
+ * a single argument. For those, modify the template ColumnUnaryFunc.txt
+ * and expand the template to generate needed classes.
+ */
+public abstract class MathFuncLongToLong extends VectorExpression {
+  private static final long serialVersionUID = 1L;
+
+  private int colNum;
+  private int outputColumn;
+
+  // Subclasses must override this with a function that implements the desired logic.
+  protected abstract long func(long d);
+
+  public MathFuncLongToLong(int colNum, int outputColumn) {
+    this.colNum = colNum;
+    this.outputColumn = outputColumn;
+  }
+
+  public MathFuncLongToLong() {
+  }
+
+  @Override
+  public void evaluate(VectorizedRowBatch batch) {
+
+    if (childExpressions != null) {
+      this.evaluateChildren(batch);
+    }
+
+    LongColumnVector inputColVector = (LongColumnVector) batch.cols[colNum];
+    LongColumnVector outputColVector = (LongColumnVector) batch.cols[outputColumn];
+    int[] sel = batch.selected;
+    boolean[] inputIsNull = inputColVector.isNull;
+    boolean[] outputIsNull = outputColVector.isNull;
+    outputColVector.noNulls = inputColVector.noNulls;
+    int n = batch.size;
+    long[] vector = inputColVector.vector;
+    long[] outputVector = outputColVector.vector;
+
+    // return immediately if batch is empty
+    if (n == 0) {
+      return;
+    }
+
+    if (inputColVector.isRepeating) {
+      outputVector[0] = func(vector[0]);
+
+      // Even if there are no nulls, we always copy over entry 0. Simplifies code.
+      outputIsNull[0] = inputIsNull[0];
+      outputColVector.isRepeating = true;
+    } else if (inputColVector.noNulls) {
+      if (batch.selectedInUse) {
+        for(int j = 0; j != n; j++) {
+          int i = sel[j];
+          outputVector[i] = func(vector[i]);
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = func(vector[i]);
+        }
+      }
+      outputColVector.isRepeating = false;
+    } else /* there are nulls */ {
+      if (batch.selectedInUse) {
+        for(int j = 0; j != n; j++) {
+          int i = sel[j];
+          outputVector[i] = func(vector[i]);
+          outputIsNull[i] = inputIsNull[i];
+      }
+      } else {
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = func(vector[i]);
+        }
+        System.arraycopy(inputIsNull, 0, outputIsNull, 0, n);
+      }
+      outputColVector.isRepeating = false;
+    }
+  }
+
+  @Override
+  public int getOutputColumn() {
+    return outputColumn;
+  }
+
+  public void setOutputColumn(int outputColumn) {
+    this.outputColumn = outputColumn;
+  }
+
+  public int getColNum() {
+    return colNum;
+  }
+
+  public void setColNum(int colNum) {
+    this.colNum = colNum;
+  }
+
+  @Override
+  public String getOutputType() {
+    return "long";
+  }
+}

Added: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/PosModDoubleToDouble.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/PosModDoubleToDouble.java?rev=1533106&view=auto
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/PosModDoubleToDouble.java (added)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/PosModDoubleToDouble.java Thu Oct 17 15:12:09 2013
@@ -0,0 +1,53 @@
+/**
+ * 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;
+
+public class PosModDoubleToDouble extends MathFuncDoubleToDouble
+    implements ISetDoubleArg {
+  private static final long serialVersionUID = 1L;
+  private double divisor;
+
+  public PosModDoubleToDouble(int inputCol, int outputCol) {
+    super(inputCol, outputCol);
+  }
+
+  public PosModDoubleToDouble() {
+    super();
+  }
+
+  @Override
+  protected double func(double v) {
+
+    // return positive modulo
+    return ((v % divisor) + divisor) % divisor;
+  }
+
+  @Override
+  public void setArg(double arg) {
+    this.divisor = arg;
+  }
+
+  public void setDivisor(double v) {
+    this.divisor = v;
+  }
+
+  public double getDivisor() {
+    return divisor;
+  }
+}

Added: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/PosModLongToLong.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/PosModLongToLong.java?rev=1533106&view=auto
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/PosModLongToLong.java (added)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/PosModLongToLong.java Thu Oct 17 15:12:09 2013
@@ -0,0 +1,53 @@
+/**
+ * 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;
+
+public class PosModLongToLong extends MathFuncLongToLong
+    implements ISetLongArg {
+  private static final long serialVersionUID = 1L;
+  private long divisor;
+
+  public PosModLongToLong(int inputCol, int outputCol) {
+    super(inputCol, outputCol);
+  }
+
+  public PosModLongToLong() {
+    super();
+  }
+
+  @Override
+  protected long func(long v) {
+
+    // return positive modulo
+    return ((v % divisor) + divisor) % divisor;
+  }
+
+  @Override
+  public void setArg(long arg) {
+    this.divisor = arg;
+  }
+
+  public void setDivisor(long v) {
+    this.divisor = v;
+  }
+
+  public long getDivisor() {
+    return divisor;
+  }
+}

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/Vectorizer.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/Vectorizer.java?rev=1533106&r1=1533105&r2=1533106&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/Vectorizer.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/Vectorizer.java Thu Oct 17 15:12:09 2013
@@ -96,6 +96,7 @@ import org.apache.hadoop.hive.ql.udf.UDF
 import org.apache.hadoop.hive.ql.udf.UDFOPNegative;
 import org.apache.hadoop.hive.ql.udf.UDFOPPlus;
 import org.apache.hadoop.hive.ql.udf.UDFOPPositive;
+import org.apache.hadoop.hive.ql.udf.UDFPosMod;
 import org.apache.hadoop.hive.ql.udf.UDFPower;
 import org.apache.hadoop.hive.ql.udf.UDFRTrim;
 import org.apache.hadoop.hive.ql.udf.UDFRadians;
@@ -116,6 +117,7 @@ import org.apache.hadoop.hive.ql.udf.UDF
 import org.apache.hadoop.hive.ql.udf.UDFToShort;
 import org.apache.hadoop.hive.ql.udf.UDFToString;
 import org.apache.hadoop.hive.ql.udf.UDFTrim;
+import org.apache.hadoop.hive.ql.udf.UDFUnhex;
 import org.apache.hadoop.hive.ql.udf.UDFWeekOfYear;
 import org.apache.hadoop.hive.ql.udf.UDFYear;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDF;
@@ -217,6 +219,7 @@ public class Vectorizer implements Physi
     supportedGenericUDFs.add(UDFLog10.class);
     supportedGenericUDFs.add(UDFLog.class);
     supportedGenericUDFs.add(UDFPower.class);
+    supportedGenericUDFs.add(UDFPosMod.class);
     supportedGenericUDFs.add(UDFRound.class);
     supportedGenericUDFs.add(UDFSqrt.class);
     supportedGenericUDFs.add(UDFSign.class);

Modified: hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorMathFunctions.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorMathFunctions.java?rev=1533106&r1=1533105&r2=1533106&view=diff
==============================================================================
--- hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorMathFunctions.java (original)
+++ hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorMathFunctions.java Thu Oct 17 15:12:09 2013
@@ -428,6 +428,30 @@ public class TestVectorMathFunctions {
   }
 
   @Test
+  public void testVectorPosMod() {
+
+    // test double->double version
+    VectorizedRowBatch b = getVectorizedRowBatchDoubleInDoubleOut();
+    DoubleColumnVector inV = (DoubleColumnVector) b.cols[0];
+    DoubleColumnVector resultV = (DoubleColumnVector) b.cols[1];
+    b.cols[0].noNulls = true;
+    inV.vector[4] = -4.0;
+    VectorExpression expr = new PosModDoubleToDouble(0, 1);
+    ((ISetDoubleArg) expr).setArg(0.3d);  // set base
+    expr.evaluate(b);
+    Assert.assertTrue(equalsWithinTolerance(((-4.0d % 0.3d) + 0.3d) % 0.3d, resultV.vector[4]));
+
+    // test long->long version
+    b = getVectorizedRowBatchLongInLongOut();
+    LongColumnVector resV2 = (LongColumnVector) b.cols[1];
+    b.cols[0].noNulls = true;
+    expr = new PosModLongToLong(0, 1);
+    ((ISetLongArg) expr).setArg(3);
+    expr.evaluate(b);
+    Assert.assertEquals(((-2 % 3) + 3) % 3, resV2.vector[0]);
+  }
+
+  @Test
   public void testVectorPower() {
     VectorizedRowBatch b = getVectorizedRowBatchDoubleInDoubleOut();
     DoubleColumnVector resultV = (DoubleColumnVector) b.cols[1];

Added: hive/trunk/ql/src/test/queries/clientpositive/vectorized_casts.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientpositive/vectorized_casts.q?rev=1533106&view=auto
==============================================================================
--- hive/trunk/ql/src/test/queries/clientpositive/vectorized_casts.q (added)
+++ hive/trunk/ql/src/test/queries/clientpositive/vectorized_casts.q Thu Oct 17 15:12:09 2013
@@ -0,0 +1,149 @@
+SET hive.vectorized.execution.enabled = true;
+
+-- Test type casting in vectorized mode to verify end-to-end functionality.
+
+explain 
+select 
+-- to boolean
+   cast (ctinyint as boolean)
+  ,cast (csmallint as boolean)
+  ,cast (cint as boolean)
+  ,cast (cbigint as boolean)
+  ,cast (cfloat as boolean)
+  ,cast (cdouble as boolean)
+  ,cast (cboolean1 as boolean)
+  ,cast (cbigint * 0 as boolean)
+  ,cast (ctimestamp1 as boolean)
+  ,cast (cstring1 as boolean)
+-- to int family
+  ,cast (ctinyint as int)
+  ,cast (csmallint as int)
+  ,cast (cint as int)
+  ,cast (cbigint as int)
+  ,cast (cfloat as int)
+  ,cast (cdouble as int)
+  ,cast (cboolean1 as int)
+  ,cast (ctimestamp1 as int)
+  ,cast (cstring1 as int)
+  ,cast (substr(cstring1, 1, 1) as int)
+  ,cast (cfloat as tinyint)
+  ,cast (cfloat as smallint)
+  ,cast (cfloat as bigint)
+-- to float family
+  ,cast (ctinyint as double)
+  ,cast (csmallint as double)
+  ,cast (cint as double)
+  ,cast (cbigint as double)
+  ,cast (cfloat as double)
+  ,cast (cdouble as double)
+  ,cast (cboolean1 as double)
+  ,cast (ctimestamp1 as double)
+  ,cast (cstring1 as double)
+  ,cast (substr(cstring1, 1, 1) as double)
+  ,cast (cint as float)
+  ,cast (cdouble as float)
+-- to timestamp
+  ,cast (ctinyint as timestamp)
+  ,cast (csmallint as timestamp)
+  ,cast (cint as timestamp)
+  ,cast (cbigint as timestamp)
+  ,cast (cfloat as timestamp)
+  ,cast (cdouble as timestamp)
+  ,cast (cboolean1 as timestamp)
+  ,cast (cbigint * 0 as timestamp)
+  ,cast (ctimestamp1 as timestamp)
+  ,cast (cstring1 as timestamp)
+  ,cast (substr(cstring1, 1, 1) as timestamp)
+-- to string
+  ,cast (ctinyint as string)
+  ,cast (csmallint as string)
+  ,cast (cint as string)
+  ,cast (cbigint as string)
+  ,cast (cfloat as string)
+  ,cast (cdouble as string)
+  ,cast (cboolean1 as string)
+  ,cast (cbigint * 0 as string)
+  ,cast (ctimestamp1 as string)
+  ,cast (cstring1 as string)
+-- nested and expression arguments
+  ,cast (cast (cfloat as int) as float)
+  ,cast (cint * 2 as double)
+  ,cast (sin(cfloat) as string)
+  ,cast (cint as float) + cast(cboolean1 as double)
+from alltypesorc
+-- limit output to a reasonably small number of rows
+where cbigint % 250 = 0;
+
+
+select 
+-- to boolean
+   cast (ctinyint as boolean)
+  ,cast (csmallint as boolean)
+  ,cast (cint as boolean)
+  ,cast (cbigint as boolean)
+  ,cast (cfloat as boolean)
+  ,cast (cdouble as boolean)
+  ,cast (cboolean1 as boolean)
+  ,cast (cbigint * 0 as boolean)
+  ,cast (ctimestamp1 as boolean)
+  ,cast (cstring1 as boolean)
+-- to int family
+  ,cast (ctinyint as int)
+  ,cast (csmallint as int)
+  ,cast (cint as int)
+  ,cast (cbigint as int)
+  ,cast (cfloat as int)
+  ,cast (cdouble as int)
+  ,cast (cboolean1 as int)
+  ,cast (ctimestamp1 as int)
+  ,cast (cstring1 as int)
+  ,cast (substr(cstring1, 1, 1) as int)
+  ,cast (cfloat as tinyint)
+  ,cast (cfloat as smallint)
+  ,cast (cfloat as bigint)
+-- to float family
+  ,cast (ctinyint as double)
+  ,cast (csmallint as double)
+  ,cast (cint as double)
+  ,cast (cbigint as double)
+  ,cast (cfloat as double)
+  ,cast (cdouble as double)
+  ,cast (cboolean1 as double)
+  ,cast (ctimestamp1 as double)
+  ,cast (cstring1 as double)
+  ,cast (substr(cstring1, 1, 1) as double)
+  ,cast (cint as float)
+  ,cast (cdouble as float)
+-- to timestamp
+  ,cast (ctinyint as timestamp)
+  ,cast (csmallint as timestamp)
+  ,cast (cint as timestamp)
+  ,cast (cbigint as timestamp)
+  ,cast (cfloat as timestamp)
+  ,cast (cdouble as timestamp)
+  ,cast (cboolean1 as timestamp)
+  ,cast (cbigint * 0 as timestamp)
+  ,cast (ctimestamp1 as timestamp)
+  ,cast (cstring1 as timestamp)
+  ,cast (substr(cstring1, 1, 1) as timestamp)
+-- to string
+  ,cast (ctinyint as string)
+  ,cast (csmallint as string)
+  ,cast (cint as string)
+  ,cast (cbigint as string)
+  ,cast (cfloat as string)
+  ,cast (cdouble as string)
+  ,cast (cboolean1 as string)
+  ,cast (cbigint * 0 as string)
+  ,cast (ctimestamp1 as string)
+  ,cast (cstring1 as string)
+-- nested and expression arguments
+  ,cast (cast (cfloat as int) as float)
+  ,cast (cint * 2 as double)
+  ,cast (sin(cfloat) as string)
+  ,cast (cint as float) + cast(cboolean1 as double)
+from alltypesorc
+-- limit output to a reasonably small number of rows
+where cbigint % 250 = 0;
+
+ 
\ No newline at end of file

Added: hive/trunk/ql/src/test/queries/clientpositive/vectorized_math_funcs.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientpositive/vectorized_math_funcs.q?rev=1533106&view=auto
==============================================================================
--- hive/trunk/ql/src/test/queries/clientpositive/vectorized_math_funcs.q (added)
+++ hive/trunk/ql/src/test/queries/clientpositive/vectorized_math_funcs.q Thu Oct 17 15:12:09 2013
@@ -0,0 +1,103 @@
+SET hive.vectorized.execution.enabled = true;
+
+-- Test math functions in vectorized mode to verify they run correctly end-to-end.
+
+explain 
+select
+   cdouble
+  ,Round(cdouble, 2)
+  ,Floor(cdouble)
+  ,Ceil(cdouble)
+  ,Rand()
+  ,Rand(98007)
+  ,Exp(ln(cdouble))
+  ,Ln(cdouble)  
+  ,Ln(cfloat)
+  ,Log10(cdouble)
+  -- Use log2 as a representative function to test all input types.
+  ,Log2(cdouble)
+  ,Log2(cfloat)
+  ,Log2(cbigint)
+  ,Log2(cint)
+  ,Log2(csmallint)
+  ,Log2(ctinyint)
+  ,Log(2.0, cdouble)
+  ,Pow(log2(cdouble), 2.0)  
+  ,Power(log2(cdouble), 2.0)
+  ,Sqrt(cdouble)
+  ,Sqrt(cbigint)
+  ,Bin(cbigint)
+  ,Hex(cdouble)
+  ,Conv(cbigint, 10, 16)
+  ,Abs(cdouble)
+  ,Abs(ctinyint)
+  ,Pmod(cint, 3)
+  ,Sin(cdouble)
+  ,Asin(cdouble)
+  ,Cos(cdouble)
+  ,ACos(cdouble)
+  ,Atan(cdouble)
+  ,Degrees(cdouble)
+  ,Radians(cdouble)
+  ,Positive(cdouble)
+  ,Positive(cbigint)
+  ,Negative(cdouble)
+  ,Sign(cdouble)
+  ,Sign(cbigint)
+  -- Test nesting
+  ,cos(-sin(log(cdouble)) + 3.14159)
+from alltypesorc
+-- limit output to a reasonably small number of rows
+where cbigint % 500 = 0
+-- test use of a math function in the WHERE clause
+and sin(cfloat) >= -1.0;
+
+select
+   cdouble
+  ,Round(cdouble, 2)
+  ,Floor(cdouble)
+  ,Ceil(cdouble)
+  -- Omit rand() from runtime test because it's nondeterministic.
+  -- ,Rand()
+  ,Rand(98007)
+  ,Exp(ln(cdouble))
+  ,Ln(cdouble)  
+  ,Ln(cfloat)
+  ,Log10(cdouble)
+  -- Use log2 as a representative function to test all input types.
+  ,Log2(cdouble)
+  ,Log2(cfloat)
+  ,Log2(cbigint)
+  ,Log2(cint)
+  ,Log2(csmallint)
+  ,Log2(ctinyint)
+  ,Log(2.0, cdouble)
+  ,Pow(log2(cdouble), 2.0)  
+  ,Power(log2(cdouble), 2.0)
+  ,Sqrt(cdouble)
+  ,Sqrt(cbigint)
+  ,Bin(cbigint)
+  ,Hex(cdouble)
+  ,Conv(cbigint, 10, 16)
+  ,Abs(cdouble)
+  ,Abs(ctinyint)
+  ,Pmod(cint, 3)
+  ,Sin(cdouble)
+  ,Asin(cdouble)
+  ,Cos(cdouble)
+  ,ACos(cdouble)
+  ,Atan(cdouble)
+  ,Degrees(cdouble)
+  ,Radians(cdouble)
+  ,Positive(cdouble)
+  ,Positive(cbigint)
+  ,Negative(cdouble)
+  ,Sign(cdouble)
+  ,Sign(cbigint)
+  -- Test nesting
+  ,cos(-sin(log(cdouble)) + 3.14159)
+from alltypesorc
+-- limit output to a reasonably small number of rows
+where cbigint % 500 = 0
+-- test use of a math function in the WHERE clause
+and sin(cfloat) >= -1.0;

Added: hive/trunk/ql/src/test/queries/clientpositive/vectorized_string_funcs.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientpositive/vectorized_string_funcs.q?rev=1533106&view=auto
==============================================================================
--- hive/trunk/ql/src/test/queries/clientpositive/vectorized_string_funcs.q (added)
+++ hive/trunk/ql/src/test/queries/clientpositive/vectorized_string_funcs.q Thu Oct 17 15:12:09 2013
@@ -0,0 +1,46 @@
+SET hive.vectorized.execution.enabled = true;
+
+-- Test string functions in vectorized mode to verify end-to-end functionality.
+
+explain 
+select 
+   substr(cstring1, 1, 2)
+  ,substr(cstring1, 2)
+  ,lower(cstring1)
+  ,upper(cstring1)
+  ,ucase(cstring1)
+  ,length(cstring1)
+  ,trim(cstring1)
+  ,ltrim(cstring1)
+  ,rtrim(cstring1)
+  ,concat(cstring1, cstring2)
+  ,concat('>', cstring1)
+  ,concat(cstring1, '<')
+  ,concat(substr(cstring1, 1, 2), substr(cstring2, 1, 2))
+from alltypesorc
+-- Limit the number of rows of output to a reasonable amount.
+where cbigint % 237 = 0
+-- Test function use in the WHERE clause.
+and length(substr(cstring1, 1, 2)) <= 2
+and cstring1 like '%';
+ 
+select 
+   substr(cstring1, 1, 2)
+  ,substr(cstring1, 2)
+  ,lower(cstring1)
+  ,upper(cstring1)
+  ,ucase(cstring1)
+  ,length(cstring1)
+  ,trim(cstring1)
+  ,ltrim(cstring1)
+  ,rtrim(cstring1)
+  ,concat(cstring1, cstring2)
+  ,concat('>', cstring1)
+  ,concat(cstring1, '<')
+  ,concat(substr(cstring1, 1, 2), substr(cstring2, 1, 2))
+from alltypesorc
+-- Limit the number of rows of output to a reasonable amount.
+where cbigint % 237 = 0
+-- Test function use in the WHERE clause.
+and length(substr(cstring1, 1, 2)) <= 2
+and cstring1 like '%';

Added: hive/trunk/ql/src/test/results/clientpositive/vectorized_casts.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientpositive/vectorized_casts.q.out?rev=1533106&view=auto
==============================================================================
--- hive/trunk/ql/src/test/results/clientpositive/vectorized_casts.q.out (added)
+++ hive/trunk/ql/src/test/results/clientpositive/vectorized_casts.q.out Thu Oct 17 15:12:09 2013
@@ -0,0 +1,477 @@
+PREHOOK: query: -- Test type casting in vectorized mode to verify end-to-end functionality.
+
+explain 
+select 
+-- to boolean
+   cast (ctinyint as boolean)
+  ,cast (csmallint as boolean)
+  ,cast (cint as boolean)
+  ,cast (cbigint as boolean)
+  ,cast (cfloat as boolean)
+  ,cast (cdouble as boolean)
+  ,cast (cboolean1 as boolean)
+  ,cast (cbigint * 0 as boolean)
+  ,cast (ctimestamp1 as boolean)
+  ,cast (cstring1 as boolean)
+-- to int family
+  ,cast (ctinyint as int)
+  ,cast (csmallint as int)
+  ,cast (cint as int)
+  ,cast (cbigint as int)
+  ,cast (cfloat as int)
+  ,cast (cdouble as int)
+  ,cast (cboolean1 as int)
+  ,cast (ctimestamp1 as int)
+  ,cast (cstring1 as int)
+  ,cast (substr(cstring1, 1, 1) as int)
+  ,cast (cfloat as tinyint)
+  ,cast (cfloat as smallint)
+  ,cast (cfloat as bigint)
+-- to float family
+  ,cast (ctinyint as double)
+  ,cast (csmallint as double)
+  ,cast (cint as double)
+  ,cast (cbigint as double)
+  ,cast (cfloat as double)
+  ,cast (cdouble as double)
+  ,cast (cboolean1 as double)
+  ,cast (ctimestamp1 as double)
+  ,cast (cstring1 as double)
+  ,cast (substr(cstring1, 1, 1) as double)
+  ,cast (cint as float)
+  ,cast (cdouble as float)
+-- to timestamp
+  ,cast (ctinyint as timestamp)
+  ,cast (csmallint as timestamp)
+  ,cast (cint as timestamp)
+  ,cast (cbigint as timestamp)
+  ,cast (cfloat as timestamp)
+  ,cast (cdouble as timestamp)
+  ,cast (cboolean1 as timestamp)
+  ,cast (cbigint * 0 as timestamp)
+  ,cast (ctimestamp1 as timestamp)
+  ,cast (cstring1 as timestamp)
+  ,cast (substr(cstring1, 1, 1) as timestamp)
+-- to string
+  ,cast (ctinyint as string)
+  ,cast (csmallint as string)
+  ,cast (cint as string)
+  ,cast (cbigint as string)
+  ,cast (cfloat as string)
+  ,cast (cdouble as string)
+  ,cast (cboolean1 as string)
+  ,cast (cbigint * 0 as string)
+  ,cast (ctimestamp1 as string)
+  ,cast (cstring1 as string)
+-- nested and expression arguments
+  ,cast (cast (cfloat as int) as float)
+  ,cast (cint * 2 as double)
+  ,cast (sin(cfloat) as string)
+  ,cast (cint as float) + cast(cboolean1 as double)
+from alltypesorc
+-- limit output to a reasonably small number of rows
+where cbigint % 250 = 0
+PREHOOK: type: QUERY
+POSTHOOK: query: -- Test type casting in vectorized mode to verify end-to-end functionality.
+
+explain 
+select 
+-- to boolean
+   cast (ctinyint as boolean)
+  ,cast (csmallint as boolean)
+  ,cast (cint as boolean)
+  ,cast (cbigint as boolean)
+  ,cast (cfloat as boolean)
+  ,cast (cdouble as boolean)
+  ,cast (cboolean1 as boolean)
+  ,cast (cbigint * 0 as boolean)
+  ,cast (ctimestamp1 as boolean)
+  ,cast (cstring1 as boolean)
+-- to int family
+  ,cast (ctinyint as int)
+  ,cast (csmallint as int)
+  ,cast (cint as int)
+  ,cast (cbigint as int)
+  ,cast (cfloat as int)
+  ,cast (cdouble as int)
+  ,cast (cboolean1 as int)
+  ,cast (ctimestamp1 as int)
+  ,cast (cstring1 as int)
+  ,cast (substr(cstring1, 1, 1) as int)
+  ,cast (cfloat as tinyint)
+  ,cast (cfloat as smallint)
+  ,cast (cfloat as bigint)
+-- to float family
+  ,cast (ctinyint as double)
+  ,cast (csmallint as double)
+  ,cast (cint as double)
+  ,cast (cbigint as double)
+  ,cast (cfloat as double)
+  ,cast (cdouble as double)
+  ,cast (cboolean1 as double)
+  ,cast (ctimestamp1 as double)
+  ,cast (cstring1 as double)
+  ,cast (substr(cstring1, 1, 1) as double)
+  ,cast (cint as float)
+  ,cast (cdouble as float)
+-- to timestamp
+  ,cast (ctinyint as timestamp)
+  ,cast (csmallint as timestamp)
+  ,cast (cint as timestamp)
+  ,cast (cbigint as timestamp)
+  ,cast (cfloat as timestamp)
+  ,cast (cdouble as timestamp)
+  ,cast (cboolean1 as timestamp)
+  ,cast (cbigint * 0 as timestamp)
+  ,cast (ctimestamp1 as timestamp)
+  ,cast (cstring1 as timestamp)
+  ,cast (substr(cstring1, 1, 1) as timestamp)
+-- to string
+  ,cast (ctinyint as string)
+  ,cast (csmallint as string)
+  ,cast (cint as string)
+  ,cast (cbigint as string)
+  ,cast (cfloat as string)
+  ,cast (cdouble as string)
+  ,cast (cboolean1 as string)
+  ,cast (cbigint * 0 as string)
+  ,cast (ctimestamp1 as string)
+  ,cast (cstring1 as string)
+-- nested and expression arguments
+  ,cast (cast (cfloat as int) as float)
+  ,cast (cint * 2 as double)
+  ,cast (sin(cfloat) as string)
+  ,cast (cint as float) + cast(cboolean1 as double)
+from alltypesorc
+-- limit output to a reasonably small number of rows
+where cbigint % 250 = 0
+POSTHOOK: type: QUERY
+ABSTRACT SYNTAX TREE:
+  (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME alltypesorc))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_FUNCTION TOK_BOOLEAN (TOK_TABLE_OR_COL ctinyint))) (TOK_SELEXPR (TOK_FUNCTION TOK_BOOLEAN (TOK_TABLE_OR_COL csmallint))) (TOK_SELEXPR (TOK_FUNCTION TOK_BOOLEAN (TOK_TABLE_OR_COL cint))) (TOK_SELEXPR (TOK_FUNCTION TOK_BOOLEAN (TOK_TABLE_OR_COL cbigint))) (TOK_SELEXPR (TOK_FUNCTION TOK_BOOLEAN (TOK_TABLE_OR_COL cfloat))) (TOK_SELEXPR (TOK_FUNCTION TOK_BOOLEAN (TOK_TABLE_OR_COL cdouble))) (TOK_SELEXPR (TOK_FUNCTION TOK_BOOLEAN (TOK_TABLE_OR_COL cboolean1))) (TOK_SELEXPR (TOK_FUNCTION TOK_BOOLEAN (* (TOK_TABLE_OR_COL cbigint) 0))) (TOK_SELEXPR (TOK_FUNCTION TOK_BOOLEAN (TOK_TABLE_OR_COL ctimestamp1))) (TOK_SELEXPR (TOK_FUNCTION TOK_BOOLEAN (TOK_TABLE_OR_COL cstring1))) (TOK_SELEXPR (TOK_FUNCTION TOK_INT (TOK_TABLE_OR_COL ctinyint))) (TOK_SELEXPR (TOK_FUNCTION TOK_INT (TOK_TABLE_OR_COL csmallint))) (TOK_SELEXPR (TOK_FUNCTION TOK_INT (TOK_TABLE_
 OR_COL cint))) (TOK_SELEXPR (TOK_FUNCTION TOK_INT (TOK_TABLE_OR_COL cbigint))) (TOK_SELEXPR (TOK_FUNCTION TOK_INT (TOK_TABLE_OR_COL cfloat))) (TOK_SELEXPR (TOK_FUNCTION TOK_INT (TOK_TABLE_OR_COL cdouble))) (TOK_SELEXPR (TOK_FUNCTION TOK_INT (TOK_TABLE_OR_COL cboolean1))) (TOK_SELEXPR (TOK_FUNCTION TOK_INT (TOK_TABLE_OR_COL ctimestamp1))) (TOK_SELEXPR (TOK_FUNCTION TOK_INT (TOK_TABLE_OR_COL cstring1))) (TOK_SELEXPR (TOK_FUNCTION TOK_INT (TOK_FUNCTION substr (TOK_TABLE_OR_COL cstring1) 1 1))) (TOK_SELEXPR (TOK_FUNCTION TOK_TINYINT (TOK_TABLE_OR_COL cfloat))) (TOK_SELEXPR (TOK_FUNCTION TOK_SMALLINT (TOK_TABLE_OR_COL cfloat))) (TOK_SELEXPR (TOK_FUNCTION TOK_BIGINT (TOK_TABLE_OR_COL cfloat))) (TOK_SELEXPR (TOK_FUNCTION TOK_DOUBLE (TOK_TABLE_OR_COL ctinyint))) (TOK_SELEXPR (TOK_FUNCTION TOK_DOUBLE (TOK_TABLE_OR_COL csmallint))) (TOK_SELEXPR (TOK_FUNCTION TOK_DOUBLE (TOK_TABLE_OR_COL cint))) (TOK_SELEXPR (TOK_FUNCTION TOK_DOUBLE (TOK_TABLE_OR_COL cbigint))) (TOK_SELEXPR (TOK_FUNCTION TOK_D
 OUBLE (TOK_TABLE_OR_COL cfloat))) (TOK_SELEXPR (TOK_FUNCTION TOK_DOUBLE (TOK_TABLE_OR_COL cdouble))) (TOK_SELEXPR (TOK_FUNCTION TOK_DOUBLE (TOK_TABLE_OR_COL cboolean1))) (TOK_SELEXPR (TOK_FUNCTION TOK_DOUBLE (TOK_TABLE_OR_COL ctimestamp1))) (TOK_SELEXPR (TOK_FUNCTION TOK_DOUBLE (TOK_TABLE_OR_COL cstring1))) (TOK_SELEXPR (TOK_FUNCTION TOK_DOUBLE (TOK_FUNCTION substr (TOK_TABLE_OR_COL cstring1) 1 1))) (TOK_SELEXPR (TOK_FUNCTION TOK_FLOAT (TOK_TABLE_OR_COL cint))) (TOK_SELEXPR (TOK_FUNCTION TOK_FLOAT (TOK_TABLE_OR_COL cdouble))) (TOK_SELEXPR (TOK_FUNCTION TOK_TIMESTAMP (TOK_TABLE_OR_COL ctinyint))) (TOK_SELEXPR (TOK_FUNCTION TOK_TIMESTAMP (TOK_TABLE_OR_COL csmallint))) (TOK_SELEXPR (TOK_FUNCTION TOK_TIMESTAMP (TOK_TABLE_OR_COL cint))) (TOK_SELEXPR (TOK_FUNCTION TOK_TIMESTAMP (TOK_TABLE_OR_COL cbigint))) (TOK_SELEXPR (TOK_FUNCTION TOK_TIMESTAMP (TOK_TABLE_OR_COL cfloat))) (TOK_SELEXPR (TOK_FUNCTION TOK_TIMESTAMP (TOK_TABLE_OR_COL cdouble))) (TOK_SELEXPR (TOK_FUNCTION TOK_TIMESTAMP (TOK_
 TABLE_OR_COL cboolean1))) (TOK_SELEXPR (TOK_FUNCTION TOK_TIMESTAMP (* (TOK_TABLE_OR_COL cbigint) 0))) (TOK_SELEXPR (TOK_FUNCTION TOK_TIMESTAMP (TOK_TABLE_OR_COL ctimestamp1))) (TOK_SELEXPR (TOK_FUNCTION TOK_TIMESTAMP (TOK_TABLE_OR_COL cstring1))) (TOK_SELEXPR (TOK_FUNCTION TOK_TIMESTAMP (TOK_FUNCTION substr (TOK_TABLE_OR_COL cstring1) 1 1))) (TOK_SELEXPR (TOK_FUNCTION TOK_STRING (TOK_TABLE_OR_COL ctinyint))) (TOK_SELEXPR (TOK_FUNCTION TOK_STRING (TOK_TABLE_OR_COL csmallint))) (TOK_SELEXPR (TOK_FUNCTION TOK_STRING (TOK_TABLE_OR_COL cint))) (TOK_SELEXPR (TOK_FUNCTION TOK_STRING (TOK_TABLE_OR_COL cbigint))) (TOK_SELEXPR (TOK_FUNCTION TOK_STRING (TOK_TABLE_OR_COL cfloat))) (TOK_SELEXPR (TOK_FUNCTION TOK_STRING (TOK_TABLE_OR_COL cdouble))) (TOK_SELEXPR (TOK_FUNCTION TOK_STRING (TOK_TABLE_OR_COL cboolean1))) (TOK_SELEXPR (TOK_FUNCTION TOK_STRING (* (TOK_TABLE_OR_COL cbigint) 0))) (TOK_SELEXPR (TOK_FUNCTION TOK_STRING (TOK_TABLE_OR_COL ctimestamp1))) (TOK_SELEXPR (TOK_FUNCTION TOK_STRING (
 TOK_TABLE_OR_COL cstring1))) (TOK_SELEXPR (TOK_FUNCTION TOK_FLOAT (TOK_FUNCTION TOK_INT (TOK_TABLE_OR_COL cfloat)))) (TOK_SELEXPR (TOK_FUNCTION TOK_DOUBLE (* (TOK_TABLE_OR_COL cint) 2))) (TOK_SELEXPR (TOK_FUNCTION TOK_STRING (TOK_FUNCTION sin (TOK_TABLE_OR_COL cfloat)))) (TOK_SELEXPR (+ (TOK_FUNCTION TOK_FLOAT (TOK_TABLE_OR_COL cint)) (TOK_FUNCTION TOK_DOUBLE (TOK_TABLE_OR_COL cboolean1))))) (TOK_WHERE (= (% (TOK_TABLE_OR_COL cbigint) 250) 0))))
+
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 is a root stage
+
+STAGE PLANS:
+  Stage: Stage-1
+    Map Reduce
+      Alias -> Map Operator Tree:
+        alltypesorc 
+          TableScan
+            alias: alltypesorc
+            Filter Operator
+              predicate:
+                  expr: ((cbigint % 250) = 0)
+                  type: boolean
+              Vectorized execution: true
+              Select Operator
+                expressions:
+                      expr: UDFToBoolean(ctinyint)
+                      type: boolean
+                      expr: UDFToBoolean(csmallint)
+                      type: boolean
+                      expr: UDFToBoolean(cint)
+                      type: boolean
+                      expr: UDFToBoolean(cbigint)
+                      type: boolean
+                      expr: UDFToBoolean(cfloat)
+                      type: boolean
+                      expr: UDFToBoolean(cdouble)
+                      type: boolean
+                      expr: cboolean1
+                      type: boolean
+                      expr: UDFToBoolean((cbigint * 0))
+                      type: boolean
+                      expr: UDFToBoolean(ctimestamp1)
+                      type: boolean
+                      expr: UDFToBoolean(cstring1)
+                      type: boolean
+                      expr: UDFToInteger(ctinyint)
+                      type: int
+                      expr: UDFToInteger(csmallint)
+                      type: int
+                      expr: cint
+                      type: int
+                      expr: UDFToInteger(cbigint)
+                      type: int
+                      expr: UDFToInteger(cfloat)
+                      type: int
+                      expr: UDFToInteger(cdouble)
+                      type: int
+                      expr: UDFToInteger(cboolean1)
+                      type: int
+                      expr: UDFToInteger(ctimestamp1)
+                      type: int
+                      expr: UDFToInteger(cstring1)
+                      type: int
+                      expr: UDFToInteger(substr(cstring1, 1, 1))
+                      type: int
+                      expr: UDFToByte(cfloat)
+                      type: tinyint
+                      expr: UDFToShort(cfloat)
+                      type: smallint
+                      expr: UDFToLong(cfloat)
+                      type: bigint
+                      expr: UDFToDouble(ctinyint)
+                      type: double
+                      expr: UDFToDouble(csmallint)
+                      type: double
+                      expr: UDFToDouble(cint)
+                      type: double
+                      expr: UDFToDouble(cbigint)
+                      type: double
+                      expr: UDFToDouble(cfloat)
+                      type: double
+                      expr: cdouble
+                      type: double
+                      expr: UDFToDouble(cboolean1)
+                      type: double
+                      expr: UDFToDouble(ctimestamp1)
+                      type: double
+                      expr: UDFToDouble(cstring1)
+                      type: double
+                      expr: UDFToDouble(substr(cstring1, 1, 1))
+                      type: double
+                      expr: UDFToFloat(cint)
+                      type: float
+                      expr: UDFToFloat(cdouble)
+                      type: float
+                      expr: CAST( ctinyint AS TIMESTAMP)
+                      type: timestamp
+                      expr: CAST( csmallint AS TIMESTAMP)
+                      type: timestamp
+                      expr: CAST( cint AS TIMESTAMP)
+                      type: timestamp
+                      expr: CAST( cbigint AS TIMESTAMP)
+                      type: timestamp
+                      expr: CAST( cfloat AS TIMESTAMP)
+                      type: timestamp
+                      expr: CAST( cdouble AS TIMESTAMP)
+                      type: timestamp
+                      expr: CAST( cboolean1 AS TIMESTAMP)
+                      type: timestamp
+                      expr: CAST( (cbigint * 0) AS TIMESTAMP)
+                      type: timestamp
+                      expr: ctimestamp1
+                      type: timestamp
+                      expr: CAST( cstring1 AS TIMESTAMP)
+                      type: timestamp
+                      expr: CAST( substr(cstring1, 1, 1) AS TIMESTAMP)
+                      type: timestamp
+                      expr: UDFToString(ctinyint)
+                      type: string
+                      expr: UDFToString(csmallint)
+                      type: string
+                      expr: UDFToString(cint)
+                      type: string
+                      expr: UDFToString(cbigint)
+                      type: string
+                      expr: UDFToString(cfloat)
+                      type: string
+                      expr: UDFToString(cdouble)
+                      type: string
+                      expr: UDFToString(cboolean1)
+                      type: string
+                      expr: UDFToString((cbigint * 0))
+                      type: string
+                      expr: UDFToString(ctimestamp1)
+                      type: string
+                      expr: cstring1
+                      type: string
+                      expr: UDFToFloat(UDFToInteger(cfloat))
+                      type: float
+                      expr: UDFToDouble((cint * 2))
+                      type: double
+                      expr: UDFToString(sin(cfloat))
+                      type: string
+                      expr: (UDFToFloat(cint) + UDFToDouble(cboolean1))
+                      type: double
+                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col19, _col20, _col21, _col22, _col23, _col24, _col25, _col26, _col27, _col28, _col29, _col30, _col31, _col32, _col33, _col34, _col35, _col36, _col37, _col38, _col39, _col40, _col41, _col42, _col43, _col44, _col45, _col46, _col47, _col48, _col49, _col50, _col51, _col52, _col53, _col54, _col55, _col56, _col57, _col58, _col59
+                Vectorized execution: true
+                File Output Operator
+                  compressed: false
+                  GlobalTableId: 0
+                  table:
+                      input format: org.apache.hadoop.mapred.TextInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                  Vectorized execution: true
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+
+
+PREHOOK: query: select 
+-- to boolean
+   cast (ctinyint as boolean)
+  ,cast (csmallint as boolean)
+  ,cast (cint as boolean)
+  ,cast (cbigint as boolean)
+  ,cast (cfloat as boolean)
+  ,cast (cdouble as boolean)
+  ,cast (cboolean1 as boolean)
+  ,cast (cbigint * 0 as boolean)
+  ,cast (ctimestamp1 as boolean)
+  ,cast (cstring1 as boolean)
+-- to int family
+  ,cast (ctinyint as int)
+  ,cast (csmallint as int)
+  ,cast (cint as int)
+  ,cast (cbigint as int)
+  ,cast (cfloat as int)
+  ,cast (cdouble as int)
+  ,cast (cboolean1 as int)
+  ,cast (ctimestamp1 as int)
+  ,cast (cstring1 as int)
+  ,cast (substr(cstring1, 1, 1) as int)
+  ,cast (cfloat as tinyint)
+  ,cast (cfloat as smallint)
+  ,cast (cfloat as bigint)
+-- to float family
+  ,cast (ctinyint as double)
+  ,cast (csmallint as double)
+  ,cast (cint as double)
+  ,cast (cbigint as double)
+  ,cast (cfloat as double)
+  ,cast (cdouble as double)
+  ,cast (cboolean1 as double)
+  ,cast (ctimestamp1 as double)
+  ,cast (cstring1 as double)
+  ,cast (substr(cstring1, 1, 1) as double)
+  ,cast (cint as float)
+  ,cast (cdouble as float)
+-- to timestamp
+  ,cast (ctinyint as timestamp)
+  ,cast (csmallint as timestamp)
+  ,cast (cint as timestamp)
+  ,cast (cbigint as timestamp)
+  ,cast (cfloat as timestamp)
+  ,cast (cdouble as timestamp)
+  ,cast (cboolean1 as timestamp)
+  ,cast (cbigint * 0 as timestamp)
+  ,cast (ctimestamp1 as timestamp)
+  ,cast (cstring1 as timestamp)
+  ,cast (substr(cstring1, 1, 1) as timestamp)
+-- to string
+  ,cast (ctinyint as string)
+  ,cast (csmallint as string)
+  ,cast (cint as string)
+  ,cast (cbigint as string)
+  ,cast (cfloat as string)
+  ,cast (cdouble as string)
+  ,cast (cboolean1 as string)
+  ,cast (cbigint * 0 as string)
+  ,cast (ctimestamp1 as string)
+  ,cast (cstring1 as string)
+-- nested and expression arguments
+  ,cast (cast (cfloat as int) as float)
+  ,cast (cint * 2 as double)
+  ,cast (sin(cfloat) as string)
+  ,cast (cint as float) + cast(cboolean1 as double)
+from alltypesorc
+-- limit output to a reasonably small number of rows
+where cbigint % 250 = 0
+PREHOOK: type: QUERY
+PREHOOK: Input: default@alltypesorc
+#### A masked pattern was here ####
+POSTHOOK: query: select 
+-- to boolean
+   cast (ctinyint as boolean)
+  ,cast (csmallint as boolean)
+  ,cast (cint as boolean)
+  ,cast (cbigint as boolean)
+  ,cast (cfloat as boolean)
+  ,cast (cdouble as boolean)
+  ,cast (cboolean1 as boolean)
+  ,cast (cbigint * 0 as boolean)
+  ,cast (ctimestamp1 as boolean)
+  ,cast (cstring1 as boolean)
+-- to int family
+  ,cast (ctinyint as int)
+  ,cast (csmallint as int)
+  ,cast (cint as int)
+  ,cast (cbigint as int)
+  ,cast (cfloat as int)
+  ,cast (cdouble as int)
+  ,cast (cboolean1 as int)
+  ,cast (ctimestamp1 as int)
+  ,cast (cstring1 as int)
+  ,cast (substr(cstring1, 1, 1) as int)
+  ,cast (cfloat as tinyint)
+  ,cast (cfloat as smallint)
+  ,cast (cfloat as bigint)
+-- to float family
+  ,cast (ctinyint as double)
+  ,cast (csmallint as double)
+  ,cast (cint as double)
+  ,cast (cbigint as double)
+  ,cast (cfloat as double)
+  ,cast (cdouble as double)
+  ,cast (cboolean1 as double)
+  ,cast (ctimestamp1 as double)
+  ,cast (cstring1 as double)
+  ,cast (substr(cstring1, 1, 1) as double)
+  ,cast (cint as float)
+  ,cast (cdouble as float)
+-- to timestamp
+  ,cast (ctinyint as timestamp)
+  ,cast (csmallint as timestamp)
+  ,cast (cint as timestamp)
+  ,cast (cbigint as timestamp)
+  ,cast (cfloat as timestamp)
+  ,cast (cdouble as timestamp)
+  ,cast (cboolean1 as timestamp)
+  ,cast (cbigint * 0 as timestamp)
+  ,cast (ctimestamp1 as timestamp)
+  ,cast (cstring1 as timestamp)
+  ,cast (substr(cstring1, 1, 1) as timestamp)
+-- to string
+  ,cast (ctinyint as string)
+  ,cast (csmallint as string)
+  ,cast (cint as string)
+  ,cast (cbigint as string)
+  ,cast (cfloat as string)
+  ,cast (cdouble as string)
+  ,cast (cboolean1 as string)
+  ,cast (cbigint * 0 as string)
+  ,cast (ctimestamp1 as string)
+  ,cast (cstring1 as string)
+-- nested and expression arguments
+  ,cast (cast (cfloat as int) as float)
+  ,cast (cint * 2 as double)
+  ,cast (sin(cfloat) as string)
+  ,cast (cint as float) + cast(cboolean1 as double)
+from alltypesorc
+-- limit output to a reasonably small number of rows
+where cbigint % 250 = 0
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@alltypesorc
+#### A masked pattern was here ####
+true	true	NULL	true	true	true	NULL	false	true	NULL	-36	-200	NULL	-2006216750	-36	-200	NULL	-14	NULL	NULL	-36	-36	-36	-36.0	-200.0	NULL	-2.00621675E9	-36.0	-200.0	NULL	-14.252	NULL	NULL	NULL	-200.0	1969-12-31 15:59:59.964	1969-12-31 15:59:59.8	NULL	1969-12-08 10:43:03.25	1969-12-31 15:59:24	1969-12-31 15:56:40	NULL	1969-12-31 16:00:00	1969-12-31 15:59:45.748	NULL	NULL	-36	-200	NULL	-2006216750	-36.0	-200.0	NULL	0	1969-12-31 15:59:45.748	NULL	-36.0	NULL	0.9917788534431158	NULL
+true	true	NULL	true	true	true	NULL	false	true	NULL	-36	-200	NULL	1599879000	-36	-200	NULL	-6	NULL	NULL	-36	-36	-36	-36.0	-200.0	NULL	1.599879E9	-36.0	-200.0	NULL	-6.183	NULL	NULL	NULL	-200.0	1969-12-31 15:59:59.964	1969-12-31 15:59:59.8	NULL	1970-01-19 04:24:39	1969-12-31 15:59:24	1969-12-31 15:56:40	NULL	1969-12-31 16:00:00	1969-12-31 15:59:53.817	NULL	NULL	-36	-200	NULL	1599879000	-36.0	-200.0	NULL	0	1969-12-31 15:59:53.817	NULL	-36.0	NULL	0.9917788534431158	NULL
+true	true	NULL	true	true	true	NULL	false	true	NULL	-30	-200	NULL	1429852250	-30	-200	NULL	12	NULL	NULL	-30	-30	-30	-30.0	-200.0	NULL	1.42985225E9	-30.0	-200.0	NULL	12.935	NULL	NULL	NULL	-200.0	1969-12-31 15:59:59.97	1969-12-31 15:59:59.8	NULL	1970-01-17 05:10:52.25	1969-12-31 15:59:30	1969-12-31 15:56:40	NULL	1969-12-31 16:00:00	1969-12-31 16:00:12.935	NULL	NULL	-30	-200	NULL	1429852250	-30.0	-200.0	NULL	0	1969-12-31 16:00:12.935	NULL	-30.0	NULL	0.9880316240928618	NULL
+true	NULL	true	true	true	NULL	false	false	true	true	-51	NULL	773600971	1053923250	-51	NULL	0	8	NULL	2	-51	-51	-51	-51.0	NULL	7.73600971E8	1.05392325E9	-51.0	NULL	0.0	8.451	NULL	2.0	7.7360096E8	NULL	1969-12-31 15:59:59.949	NULL	1970-01-09 14:53:20.971	1970-01-12 20:45:23.25	1969-12-31 15:59:09	NULL	1969-12-31 16:00:00	1969-12-31 16:00:00	1969-12-31 16:00:08.451	NULL	NULL	-51	NULL	773600971	1053923250	-51.0	NULL	FALSE	0	1969-12-31 16:00:08.451	2yK4Bx76O	-51.0	1.547201942E9	-0.6702291758433747	7.73600971E8
+true	NULL	true	true	true	NULL	true	false	true	true	-51	NULL	747553882	-1930467250	-51	NULL	1	8	NULL	NULL	-51	-51	-51	-51.0	NULL	7.47553882E8	-1.93046725E9	-51.0	NULL	1.0	8.451	NULL	NULL	7.4755386E8	NULL	1969-12-31 15:59:59.949	NULL	1970-01-09 07:39:13.882	1969-12-09 07:45:32.75	1969-12-31 15:59:09	NULL	1969-12-31 16:00:00.001	1969-12-31 16:00:00	1969-12-31 16:00:08.451	NULL	NULL	-51	NULL	747553882	-1930467250	-51.0	NULL	TRUE	0	1969-12-31 16:00:08.451	q8M86Fx0r	-51.0	1.495107764E9	-0.6702291758433747	7.47553883E8
+true	true	NULL	true	true	true	NULL	false	true	NULL	20	15601	NULL	-362433250	20	15601	NULL	-14	NULL	NULL	20	20	20	20.0	15601.0	NULL	-3.6243325E8	20.0	15601.0	NULL	-14.871	NULL	NULL	NULL	15601.0	1969-12-31 16:00:00.02	1969-12-31 16:00:15.601	NULL	1969-12-27 11:19:26.75	1969-12-31 16:00:20	1969-12-31 20:20:01	NULL	1969-12-31 16:00:00	1969-12-31 15:59:45.129	NULL	NULL	20	15601	NULL	-362433250	20.0	15601.0	NULL	0	1969-12-31 15:59:45.129	NULL	20.0	NULL	0.9129452507276277	NULL
+true	true	NULL	true	true	true	NULL	false	true	NULL	-38	15601	NULL	-1858689000	-38	15601	NULL	-1	NULL	NULL	-38	-38	-38	-38.0	15601.0	NULL	-1.858689E9	-38.0	15601.0	NULL	-1.386	NULL	NULL	NULL	15601.0	1969-12-31 15:59:59.962	1969-12-31 16:00:15.601	NULL	1969-12-10 03:41:51	1969-12-31 15:59:22	1969-12-31 20:20:01	NULL	1969-12-31 16:00:00	1969-12-31 15:59:58.614	NULL	NULL	-38	15601	NULL	-1858689000	-38.0	15601.0	NULL	0	1969-12-31 15:59:58.614	NULL	-38.0	NULL	-0.2963685787093853	NULL
+true	true	NULL	true	true	true	NULL	false	true	NULL	-5	15601	NULL	612416000	-5	15601	NULL	4	NULL	NULL	-5	-5	-5	-5.0	15601.0	NULL	6.12416E8	-5.0	15601.0	NULL	4.679	NULL	NULL	NULL	15601.0	1969-12-31 15:59:59.995	1969-12-31 16:00:15.601	NULL	1970-01-07 18:06:56	1969-12-31 15:59:55	1969-12-31 20:20:01	NULL	1969-12-31 16:00:00	1969-12-31 16:00:04.679	NULL	NULL	-5	15601	NULL	612416000	-5.0	15601.0	NULL	0	1969-12-31 16:00:04.679	NULL	-5.0	NULL	0.9589242746631385	NULL
+true	true	NULL	true	true	true	NULL	false	true	NULL	48	15601	NULL	-795361000	48	15601	NULL	-9	NULL	NULL	48	48	48	48.0	15601.0	NULL	-7.95361E8	48.0	15601.0	NULL	-9.765	NULL	NULL	NULL	15601.0	1969-12-31 16:00:00.048	1969-12-31 16:00:15.601	NULL	1969-12-22 11:03:59	1969-12-31 16:00:48	1969-12-31 20:20:01	NULL	1969-12-31 16:00:00	1969-12-31 15:59:50.235	NULL	NULL	48	15601	NULL	-795361000	48.0	15601.0	NULL	0	1969-12-31 15:59:50.235	NULL	48.0	NULL	-0.7682546613236668	NULL
+true	NULL	true	true	true	NULL	false	false	true	true	8	NULL	-661621138	-931392750	8	NULL	0	15	NULL	NULL	8	8	8	8.0	NULL	-6.61621138E8	-9.3139275E8	8.0	NULL	0.0	15.892	NULL	NULL	-6.6162112E8	NULL	1969-12-31 16:00:00.008	NULL	1969-12-24 00:12:58.862	1969-12-20 21:16:47.25	1969-12-31 16:00:08	NULL	1969-12-31 16:00:00	1969-12-31 16:00:00	1969-12-31 16:00:15.892	NULL	NULL	8	NULL	-661621138	-931392750	8.0	NULL	FALSE	0	1969-12-31 16:00:15.892	L15l8i5k558tBcDV20	8.0	-1.323242276E9	0.9893582466233818	-6.61621138E8
+true	NULL	true	true	true	NULL	false	false	true	true	8	NULL	-102936434	-1312782750	8	NULL	0	15	NULL	NULL	8	8	8	8.0	NULL	-1.02936434E8	-1.31278275E9	8.0	NULL	0.0	15.892	NULL	NULL	-1.02936432E8	NULL	1969-12-31 16:00:00.008	NULL	1969-12-30 11:24:23.566	1969-12-16 11:20:17.25	1969-12-31 16:00:08	NULL	1969-12-31 16:00:00	1969-12-31 16:00:00	1969-12-31 16:00:15.892	NULL	NULL	8	NULL	-102936434	-1312782750	8.0	NULL	FALSE	0	1969-12-31 16:00:15.892	eJROSNhugc3kQR7Pb	8.0	-2.05872868E8	0.9893582466233818	-1.02936434E8
+true	NULL	true	true	true	NULL	false	false	true	true	8	NULL	805179664	868161500	8	NULL	0	15	NULL	NULL	8	8	8	8.0	NULL	8.05179664E8	8.681615E8	8.0	NULL	0.0	15.892	NULL	NULL	8.0517965E8	NULL	1969-12-31 16:00:00.008	NULL	1970-01-09 23:39:39.664	1970-01-10 17:09:21.5	1969-12-31 16:00:08	NULL	1969-12-31 16:00:00	1969-12-31 16:00:00	1969-12-31 16:00:15.892	NULL	NULL	8	NULL	805179664	868161500	8.0	NULL	FALSE	0	1969-12-31 16:00:15.892	e005B5q	8.0	1.610359328E9	0.9893582466233818	8.05179664E8
+true	NULL	true	true	true	NULL	false	false	true	true	8	NULL	-669632311	1588591250	8	NULL	0	15	NULL	3	8	8	8	8.0	NULL	-6.69632311E8	1.58859125E9	8.0	NULL	0.0	15.892	NULL	3.0	-6.6963232E8	NULL	1969-12-31 16:00:00.008	NULL	1969-12-23 21:59:27.689	1970-01-19 01:16:31.25	1969-12-31 16:00:08	NULL	1969-12-31 16:00:00	1969-12-31 16:00:00	1969-12-31 16:00:15.892	NULL	NULL	8	NULL	-669632311	1588591250	8.0	NULL	FALSE	0	1969-12-31 16:00:15.892	3r3sDvfUkG0yTP3LnX5mNQRr	8.0	-1.339264622E9	0.9893582466233818	-6.69632311E8
+true	NULL	true	true	true	NULL	true	false	true	true	8	NULL	890988972	-1862301000	8	NULL	1	15	NULL	NULL	8	8	8	8.0	NULL	8.90988972E8	-1.862301E9	8.0	NULL	1.0	15.892	NULL	NULL	8.9098899E8	NULL	1969-12-31 16:00:00.008	NULL	1970-01-10 23:29:48.972	1969-12-10 02:41:39	1969-12-31 16:00:08	NULL	1969-12-31 16:00:00.001	1969-12-31 16:00:00	1969-12-31 16:00:15.892	NULL	NULL	8	NULL	890988972	-1862301000	8.0	NULL	TRUE	0	1969-12-31 16:00:15.892	XylAH4	8.0	1.781977944E9	0.9893582466233818	8.90988973E8
+true	NULL	true	true	true	NULL	true	false	true	true	8	NULL	930867246	1205399250	8	NULL	1	15	NULL	NULL	8	8	8	8.0	NULL	9.30867246E8	1.20539925E9	8.0	NULL	1.0	15.892	NULL	NULL	9.3086726E8	NULL	1969-12-31 16:00:00.008	NULL	1970-01-11 10:34:27.246	1970-01-14 14:49:59.25	1969-12-31 16:00:08	NULL	1969-12-31 16:00:00.001	1969-12-31 16:00:00	1969-12-31 16:00:15.892	NULL	NULL	8	NULL	930867246	1205399250	8.0	NULL	TRUE	0	1969-12-31 16:00:15.892	c1V8o1A	8.0	1.861734492E9	0.9893582466233818	9.30867247E8
+true	true	NULL	true	true	true	NULL	false	true	NULL	-59	-7196	NULL	-1604890000	-59	-7196	NULL	13	NULL	NULL	-59	-59	-59	-59.0	-7196.0	NULL	-1.60489E9	-59.0	-7196.0	NULL	13.15	NULL	NULL	NULL	-7196.0	1969-12-31 15:59:59.941	1969-12-31 15:59:52.804	NULL	1969-12-13 02:11:50	1969-12-31 15:59:01	1969-12-31 14:00:04	NULL	1969-12-31 16:00:00	1969-12-31 16:00:13.15	NULL	NULL	-59	-7196	NULL	-1604890000	-59.0	-7196.0	NULL	0	1969-12-31 16:00:13.15	NULL	-59.0	NULL	-0.6367380071391379	NULL
+true	true	NULL	true	true	true	NULL	false	true	NULL	-21	-7196	NULL	1542429000	-21	-7196	NULL	-4	NULL	NULL	-21	-21	-21	-21.0	-7196.0	NULL	1.542429E9	-21.0	-7196.0	NULL	-4.1	NULL	NULL	NULL	-7196.0	1969-12-31 15:59:59.979	1969-12-31 15:59:52.804	NULL	1970-01-18 12:27:09	1969-12-31 15:59:39	1969-12-31 14:00:04	NULL	1969-12-31 16:00:00	1969-12-31 15:59:55.9	NULL	NULL	-21	-7196	NULL	1542429000	-21.0	-7196.0	NULL	0	1969-12-31 15:59:55.9	NULL	-21.0	NULL	-0.8366556385360561	NULL
+true	true	NULL	true	true	true	NULL	false	true	NULL	-60	-7196	NULL	1516314750	-60	-7196	NULL	-7	NULL	NULL	-60	-60	-60	-60.0	-7196.0	NULL	1.51631475E9	-60.0	-7196.0	NULL	-7.592	NULL	NULL	NULL	-7196.0	1969-12-31 15:59:59.94	1969-12-31 15:59:52.804	NULL	1970-01-18 05:11:54.75	1969-12-31 15:59:00	1969-12-31 14:00:04	NULL	1969-12-31 16:00:00	1969-12-31 15:59:52.408	NULL	NULL	-60	-7196	NULL	1516314750	-60.0	-7196.0	NULL	0	1969-12-31 15:59:52.408	NULL	-60.0	NULL	0.3048106211022167	NULL
+true	true	NULL	true	true	true	NULL	false	true	NULL	-14	-7196	NULL	-1552199500	-14	-7196	NULL	11	NULL	NULL	-14	-14	-14	-14.0	-7196.0	NULL	-1.5521995E9	-14.0	-7196.0	NULL	11.065	NULL	NULL	NULL	-7196.0	1969-12-31 15:59:59.986	1969-12-31 15:59:52.804	NULL	1969-12-13 16:50:00.5	1969-12-31 15:59:46	1969-12-31 14:00:04	NULL	1969-12-31 16:00:00	1969-12-31 16:00:11.065	NULL	NULL	-14	-7196	NULL	-1552199500	-14.0	-7196.0	NULL	0	1969-12-31 16:00:11.065	NULL	-14.0	NULL	-0.9906073556948704	NULL
+true	true	NULL	true	true	true	NULL	false	true	NULL	59	-7196	NULL	-1137754500	59	-7196	NULL	10	NULL	NULL	59	59	59	59.0	-7196.0	NULL	-1.1377545E9	59.0	-7196.0	NULL	10.956	NULL	NULL	NULL	-7196.0	1969-12-31 16:00:00.059	1969-12-31 15:59:52.804	NULL	1969-12-18 11:57:25.5	1969-12-31 16:00:59	1969-12-31 14:00:04	NULL	1969-12-31 16:00:00	1969-12-31 16:00:10.956	NULL	NULL	59	-7196	NULL	-1137754500	59.0	-7196.0	NULL	0	1969-12-31 16:00:10.956	NULL	59.0	NULL	0.6367380071391379	NULL
+true	true	NULL	true	true	true	NULL	false	true	NULL	-8	-7196	NULL	-1849991500	-8	-7196	NULL	3	NULL	NULL	-8	-8	-8	-8.0	-7196.0	NULL	-1.8499915E9	-8.0	-7196.0	NULL	3.136	NULL	NULL	NULL	-7196.0	1969-12-31 15:59:59.992	1969-12-31 15:59:52.804	NULL	1969-12-10 06:06:48.5	1969-12-31 15:59:52	1969-12-31 14:00:04	NULL	1969-12-31 16:00:00	1969-12-31 16:00:03.136	NULL	NULL	-8	-7196	NULL	-1849991500	-8.0	-7196.0	NULL	0	1969-12-31 16:00:03.136	NULL	-8.0	NULL	-0.9893582466233818	NULL
+true	true	NULL	true	true	true	NULL	false	true	NULL	5	-7196	NULL	-1015607500	5	-7196	NULL	10	NULL	NULL	5	5	5	5.0	-7196.0	NULL	-1.0156075E9	5.0	-7196.0	NULL	10.973	NULL	NULL	NULL	-7196.0	1969-12-31 16:00:00.005	1969-12-31 15:59:52.804	NULL	1969-12-19 21:53:12.5	1969-12-31 16:00:05	1969-12-31 14:00:04	NULL	1969-12-31 16:00:00	1969-12-31 16:00:10.973	NULL	NULL	5	-7196	NULL	-1015607500	5.0	-7196.0	NULL	0	1969-12-31 16:00:10.973	NULL	5.0	NULL	-0.9589242746631385	NULL
+true	true	NULL	true	true	true	NULL	false	true	NULL	-24	-7196	NULL	829111000	-24	-7196	NULL	-6	NULL	NULL	-24	-24	-24	-24.0	-7196.0	NULL	8.29111E8	-24.0	-7196.0	NULL	-6.855	NULL	NULL	NULL	-7196.0	1969-12-31 15:59:59.976	1969-12-31 15:59:52.804	NULL	1970-01-10 06:18:31	1969-12-31 15:59:36	1969-12-31 14:00:04	NULL	1969-12-31 16:00:00	1969-12-31 15:59:53.145	NULL	NULL	-24	-7196	NULL	829111000	-24.0	-7196.0	NULL	0	1969-12-31 15:59:53.145	NULL	-24.0	NULL	0.9055783620066238	NULL
+true	true	NULL	true	true	true	NULL	false	true	NULL	-50	-7196	NULL	-1031187250	-50	-7196	NULL	-5	NULL	NULL	-50	-50	-50	-50.0	-7196.0	NULL	-1.03118725E9	-50.0	-7196.0	NULL	-5.267	NULL	NULL	NULL	-7196.0	1969-12-31 15:59:59.95	1969-12-31 15:59:52.804	NULL	1969-12-19 17:33:32.75	1969-12-31 15:59:10	1969-12-31 14:00:04	NULL	1969-12-31 16:00:00	1969-12-31 15:59:54.733	NULL	NULL	-50	-7196	NULL	-1031187250	-50.0	-7196.0	NULL	0	1969-12-31 15:59:54.733	NULL	-50.0	NULL	0.26237485370392877	NULL
+true	NULL	true	true	true	NULL	true	false	true	true	11	NULL	-64615982	1803053750	11	NULL	1	2	NULL	8	11	11	11	11.0	NULL	-6.4615982E7	1.80305375E9	11.0	NULL	1.0	2.351	NULL	8.0	-6.4615984E7	NULL	1969-12-31 16:00:00.011	NULL	1969-12-30 22:03:04.018	1970-01-21 12:50:53.75	1969-12-31 16:00:11	NULL	1969-12-31 16:00:00.001	1969-12-31 16:00:00	1969-12-31 16:00:02.351	NULL	NULL	11	NULL	-64615982	1803053750	11.0	NULL	TRUE	0	1969-12-31 16:00:02.351	8J5OB7K26PEV7kdbeHr3	11.0	-1.29231964E8	-0.9999902065507035	-6.4615981E7
+true	NULL	true	true	true	NULL	true	false	true	true	11	NULL	-335450417	1233327000	11	NULL	1	2	NULL	NULL	11	11	11	11.0	NULL	-3.35450417E8	1.233327E9	11.0	NULL	1.0	2.351	NULL	NULL	-3.35450432E8	NULL	1969-12-31 16:00:00.011	NULL	1969-12-27 18:49:09.583	1970-01-14 22:35:27	1969-12-31 16:00:11	NULL	1969-12-31 16:00:00.001	1969-12-31 16:00:00	1969-12-31 16:00:02.351	NULL	NULL	11	NULL	-335450417	1233327000	11.0	NULL	TRUE	0	1969-12-31 16:00:02.351	dOYnqgaXoJ1P3ERwxe5N7	11.0	-6.70900834E8	-0.9999902065507035	-3.35450416E8

Added: hive/trunk/ql/src/test/results/clientpositive/vectorized_math_funcs.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientpositive/vectorized_math_funcs.q.out?rev=1533106&view=auto
==============================================================================
--- hive/trunk/ql/src/test/results/clientpositive/vectorized_math_funcs.q.out (added)
+++ hive/trunk/ql/src/test/results/clientpositive/vectorized_math_funcs.q.out Thu Oct 17 15:12:09 2013
@@ -0,0 +1,339 @@
+PREHOOK: query: -- Test math functions in vectorized mode to verify they run correctly end-to-end.
+
+explain 
+select
+   cdouble
+  ,Round(cdouble, 2)
+  ,Floor(cdouble)
+  ,Ceil(cdouble)
+  ,Rand()
+  ,Rand(98007)
+  ,Exp(ln(cdouble))
+  ,Ln(cdouble)  
+  ,Ln(cfloat)
+  ,Log10(cdouble)
+  -- Use log2 as a representative function to test all input types.
+  ,Log2(cdouble)
+  ,Log2(cfloat)
+  ,Log2(cbigint)
+  ,Log2(cint)
+  ,Log2(csmallint)
+  ,Log2(ctinyint)
+  ,Log(2.0, cdouble)
+  ,Pow(log2(cdouble), 2.0)  
+  ,Power(log2(cdouble), 2.0)
+  ,Sqrt(cdouble)
+  ,Sqrt(cbigint)
+  ,Bin(cbigint)
+  ,Hex(cdouble)
+  ,Conv(cbigint, 10, 16)
+  ,Abs(cdouble)
+  ,Abs(ctinyint)
+  ,Pmod(cint, 3)
+  ,Sin(cdouble)
+  ,Asin(cdouble)
+  ,Cos(cdouble)
+  ,ACos(cdouble)
+  ,Atan(cdouble)
+  ,Degrees(cdouble)
+  ,Radians(cdouble)
+  ,Positive(cdouble)
+  ,Positive(cbigint)
+  ,Negative(cdouble)
+  ,Sign(cdouble)
+  ,Sign(cbigint)
+  -- Test nesting
+  ,cos(-sin(log(cdouble)) + 3.14159)
+from alltypesorc
+-- limit output to a reasonably small number of rows
+where cbigint % 500 = 0
+-- test use of a math function in the WHERE clause
+and sin(cfloat) >= -1.0
+PREHOOK: type: QUERY
+POSTHOOK: query: -- Test math functions in vectorized mode to verify they run correctly end-to-end.
+
+explain 
+select
+   cdouble
+  ,Round(cdouble, 2)
+  ,Floor(cdouble)
+  ,Ceil(cdouble)
+  ,Rand()
+  ,Rand(98007)
+  ,Exp(ln(cdouble))
+  ,Ln(cdouble)  
+  ,Ln(cfloat)
+  ,Log10(cdouble)
+  -- Use log2 as a representative function to test all input types.
+  ,Log2(cdouble)
+  ,Log2(cfloat)
+  ,Log2(cbigint)
+  ,Log2(cint)
+  ,Log2(csmallint)
+  ,Log2(ctinyint)
+  ,Log(2.0, cdouble)
+  ,Pow(log2(cdouble), 2.0)  
+  ,Power(log2(cdouble), 2.0)
+  ,Sqrt(cdouble)
+  ,Sqrt(cbigint)
+  ,Bin(cbigint)
+  ,Hex(cdouble)
+  ,Conv(cbigint, 10, 16)
+  ,Abs(cdouble)
+  ,Abs(ctinyint)
+  ,Pmod(cint, 3)
+  ,Sin(cdouble)
+  ,Asin(cdouble)
+  ,Cos(cdouble)
+  ,ACos(cdouble)
+  ,Atan(cdouble)
+  ,Degrees(cdouble)
+  ,Radians(cdouble)
+  ,Positive(cdouble)
+  ,Positive(cbigint)
+  ,Negative(cdouble)
+  ,Sign(cdouble)
+  ,Sign(cbigint)
+  -- Test nesting
+  ,cos(-sin(log(cdouble)) + 3.14159)
+from alltypesorc
+-- limit output to a reasonably small number of rows
+where cbigint % 500 = 0
+-- test use of a math function in the WHERE clause
+and sin(cfloat) >= -1.0
+POSTHOOK: type: QUERY
+ABSTRACT SYNTAX TREE:
+  (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME alltypesorc))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_TABLE_OR_COL cdouble)) (TOK_SELEXPR (TOK_FUNCTION Round (TOK_TABLE_OR_COL cdouble) 2)) (TOK_SELEXPR (TOK_FUNCTION Floor (TOK_TABLE_OR_COL cdouble))) (TOK_SELEXPR (TOK_FUNCTION Ceil (TOK_TABLE_OR_COL cdouble))) (TOK_SELEXPR (TOK_FUNCTION Rand)) (TOK_SELEXPR (TOK_FUNCTION Rand 98007)) (TOK_SELEXPR (TOK_FUNCTION Exp (TOK_FUNCTION ln (TOK_TABLE_OR_COL cdouble)))) (TOK_SELEXPR (TOK_FUNCTION Ln (TOK_TABLE_OR_COL cdouble))) (TOK_SELEXPR (TOK_FUNCTION Ln (TOK_TABLE_OR_COL cfloat))) (TOK_SELEXPR (TOK_FUNCTION Log10 (TOK_TABLE_OR_COL cdouble))) (TOK_SELEXPR (TOK_FUNCTION Log2 (TOK_TABLE_OR_COL cdouble))) (TOK_SELEXPR (TOK_FUNCTION Log2 (TOK_TABLE_OR_COL cfloat))) (TOK_SELEXPR (TOK_FUNCTION Log2 (TOK_TABLE_OR_COL cbigint))) (TOK_SELEXPR (TOK_FUNCTION Log2 (TOK_TABLE_OR_COL cint))) (TOK_SELEXPR (TOK_FUNCTION Log2 (TOK_TABLE_OR_COL csmallint))) (TOK_SE
 LEXPR (TOK_FUNCTION Log2 (TOK_TABLE_OR_COL ctinyint))) (TOK_SELEXPR (TOK_FUNCTION Log 2.0 (TOK_TABLE_OR_COL cdouble))) (TOK_SELEXPR (TOK_FUNCTION Pow (TOK_FUNCTION log2 (TOK_TABLE_OR_COL cdouble)) 2.0)) (TOK_SELEXPR (TOK_FUNCTION Power (TOK_FUNCTION log2 (TOK_TABLE_OR_COL cdouble)) 2.0)) (TOK_SELEXPR (TOK_FUNCTION Sqrt (TOK_TABLE_OR_COL cdouble))) (TOK_SELEXPR (TOK_FUNCTION Sqrt (TOK_TABLE_OR_COL cbigint))) (TOK_SELEXPR (TOK_FUNCTION Bin (TOK_TABLE_OR_COL cbigint))) (TOK_SELEXPR (TOK_FUNCTION Hex (TOK_TABLE_OR_COL cdouble))) (TOK_SELEXPR (TOK_FUNCTION Conv (TOK_TABLE_OR_COL cbigint) 10 16)) (TOK_SELEXPR (TOK_FUNCTION Abs (TOK_TABLE_OR_COL cdouble))) (TOK_SELEXPR (TOK_FUNCTION Abs (TOK_TABLE_OR_COL ctinyint))) (TOK_SELEXPR (TOK_FUNCTION Pmod (TOK_TABLE_OR_COL cint) 3)) (TOK_SELEXPR (TOK_FUNCTION Sin (TOK_TABLE_OR_COL cdouble))) (TOK_SELEXPR (TOK_FUNCTION Asin (TOK_TABLE_OR_COL cdouble))) (TOK_SELEXPR (TOK_FUNCTION Cos (TOK_TABLE_OR_COL cdouble))) (TOK_SELEXPR (TOK_FUNCTION ACos (TOK_
 TABLE_OR_COL cdouble))) (TOK_SELEXPR (TOK_FUNCTION Atan (TOK_TABLE_OR_COL cdouble))) (TOK_SELEXPR (TOK_FUNCTION Degrees (TOK_TABLE_OR_COL cdouble))) (TOK_SELEXPR (TOK_FUNCTION Radians (TOK_TABLE_OR_COL cdouble))) (TOK_SELEXPR (TOK_FUNCTION Positive (TOK_TABLE_OR_COL cdouble))) (TOK_SELEXPR (TOK_FUNCTION Positive (TOK_TABLE_OR_COL cbigint))) (TOK_SELEXPR (TOK_FUNCTION Negative (TOK_TABLE_OR_COL cdouble))) (TOK_SELEXPR (TOK_FUNCTION Sign (TOK_TABLE_OR_COL cdouble))) (TOK_SELEXPR (TOK_FUNCTION Sign (TOK_TABLE_OR_COL cbigint))) (TOK_SELEXPR (TOK_FUNCTION cos (+ (- (TOK_FUNCTION sin (TOK_FUNCTION log (TOK_TABLE_OR_COL cdouble)))) 3.14159)))) (TOK_WHERE (and (= (% (TOK_TABLE_OR_COL cbigint) 500) 0) (>= (TOK_FUNCTION sin (TOK_TABLE_OR_COL cfloat)) (- 1.0))))))
+
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 is a root stage
+
+STAGE PLANS:
+  Stage: Stage-1
+    Map Reduce
+      Alias -> Map Operator Tree:
+        alltypesorc 
+          TableScan
+            alias: alltypesorc
+            Filter Operator
+              predicate:
+                  expr: (((cbigint % 500) = 0) and (sin(cfloat) >= (- 1.0)))
+                  type: boolean
+              Vectorized execution: true
+              Select Operator
+                expressions:
+                      expr: cdouble
+                      type: double
+                      expr: round(cdouble, 2)
+                      type: double
+                      expr: floor(cdouble)
+                      type: bigint
+                      expr: ceil(cdouble)
+                      type: bigint
+                      expr: rand()
+                      type: double
+                      expr: rand(98007)
+                      type: double
+                      expr: exp(ln(cdouble))
+                      type: double
+                      expr: ln(cdouble)
+                      type: double
+                      expr: ln(cfloat)
+                      type: double
+                      expr: log10(cdouble)
+                      type: double
+                      expr: log2(cdouble)
+                      type: double
+                      expr: log2(cfloat)
+                      type: double
+                      expr: log2(cbigint)
+                      type: double
+                      expr: log2(cint)
+                      type: double
+                      expr: log2(csmallint)
+                      type: double
+                      expr: log2(ctinyint)
+                      type: double
+                      expr: log(2.0, cdouble)
+                      type: double
+                      expr: pow(log2(cdouble), 2.0)
+                      type: double
+                      expr: power(log2(cdouble), 2.0)
+                      type: double
+                      expr: sqrt(cdouble)
+                      type: double
+                      expr: sqrt(cbigint)
+                      type: double
+                      expr: bin(cbigint)
+                      type: string
+                      expr: hex(cdouble)
+                      type: string
+                      expr: conv(cbigint, 10, 16)
+                      type: string
+                      expr: abs(cdouble)
+                      type: double
+                      expr: abs(ctinyint)
+                      type: int
+                      expr: pmod(cint, 3)
+                      type: int
+                      expr: sin(cdouble)
+                      type: double
+                      expr: asin(cdouble)
+                      type: double
+                      expr: cos(cdouble)
+                      type: double
+                      expr: acos(cdouble)
+                      type: double
+                      expr: atan(cdouble)
+                      type: double
+                      expr: degrees(cdouble)
+                      type: double
+                      expr: radians(cdouble)
+                      type: double
+                      expr: cdouble
+                      type: double
+                      expr: cbigint
+                      type: bigint
+                      expr: (- cdouble)
+                      type: double
+                      expr: sign(cdouble)
+                      type: double
+                      expr: sign(cbigint)
+                      type: double
+                      expr: cos(((- sin(log(cdouble))) + 3.14159))
+                      type: double
+                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col19, _col20, _col21, _col22, _col23, _col24, _col25, _col26, _col27, _col28, _col29, _col30, _col31, _col32, _col33, _col34, _col35, _col36, _col37, _col38, _col39
+                Vectorized execution: true
+                File Output Operator
+                  compressed: false
+                  GlobalTableId: 0
+                  table:
+                      input format: org.apache.hadoop.mapred.TextInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                  Vectorized execution: true
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+
+
+PREHOOK: query: select
+   cdouble
+  ,Round(cdouble, 2)
+  ,Floor(cdouble)
+  ,Ceil(cdouble)
+  -- Omit rand() from runtime test because it's nondeterministic.
+  -- ,Rand()
+  ,Rand(98007)
+  ,Exp(ln(cdouble))
+  ,Ln(cdouble)  
+  ,Ln(cfloat)
+  ,Log10(cdouble)
+  -- Use log2 as a representative function to test all input types.
+  ,Log2(cdouble)
+  ,Log2(cfloat)
+  ,Log2(cbigint)
+  ,Log2(cint)
+  ,Log2(csmallint)
+  ,Log2(ctinyint)
+  ,Log(2.0, cdouble)
+  ,Pow(log2(cdouble), 2.0)  
+  ,Power(log2(cdouble), 2.0)
+  ,Sqrt(cdouble)
+  ,Sqrt(cbigint)
+  ,Bin(cbigint)
+  ,Hex(cdouble)
+  ,Conv(cbigint, 10, 16)
+  ,Abs(cdouble)
+  ,Abs(ctinyint)
+  ,Pmod(cint, 3)
+  ,Sin(cdouble)
+  ,Asin(cdouble)
+  ,Cos(cdouble)
+  ,ACos(cdouble)
+  ,Atan(cdouble)
+  ,Degrees(cdouble)
+  ,Radians(cdouble)
+  ,Positive(cdouble)
+  ,Positive(cbigint)
+  ,Negative(cdouble)
+  ,Sign(cdouble)
+  ,Sign(cbigint)
+  -- Test nesting
+  ,cos(-sin(log(cdouble)) + 3.14159)
+from alltypesorc
+-- limit output to a reasonably small number of rows
+where cbigint % 500 = 0
+-- test use of a math function in the WHERE clause
+and sin(cfloat) >= -1.0
+PREHOOK: type: QUERY
+PREHOOK: Input: default@alltypesorc
+#### A masked pattern was here ####
+POSTHOOK: query: select
+   cdouble
+  ,Round(cdouble, 2)
+  ,Floor(cdouble)
+  ,Ceil(cdouble)
+  -- Omit rand() from runtime test because it's nondeterministic.
+  -- ,Rand()
+  ,Rand(98007)
+  ,Exp(ln(cdouble))
+  ,Ln(cdouble)  
+  ,Ln(cfloat)
+  ,Log10(cdouble)
+  -- Use log2 as a representative function to test all input types.
+  ,Log2(cdouble)
+  ,Log2(cfloat)
+  ,Log2(cbigint)
+  ,Log2(cint)
+  ,Log2(csmallint)
+  ,Log2(ctinyint)
+  ,Log(2.0, cdouble)
+  ,Pow(log2(cdouble), 2.0)  
+  ,Power(log2(cdouble), 2.0)
+  ,Sqrt(cdouble)
+  ,Sqrt(cbigint)
+  ,Bin(cbigint)
+  ,Hex(cdouble)
+  ,Conv(cbigint, 10, 16)
+  ,Abs(cdouble)
+  ,Abs(ctinyint)
+  ,Pmod(cint, 3)
+  ,Sin(cdouble)
+  ,Asin(cdouble)
+  ,Cos(cdouble)
+  ,ACos(cdouble)
+  ,Atan(cdouble)
+  ,Degrees(cdouble)
+  ,Radians(cdouble)
+  ,Positive(cdouble)
+  ,Positive(cbigint)
+  ,Negative(cdouble)
+  ,Sign(cdouble)
+  ,Sign(cbigint)
+  -- Test nesting
+  ,cos(-sin(log(cdouble)) + 3.14159)
+from alltypesorc
+-- limit output to a reasonably small number of rows
+where cbigint % 500 = 0
+-- test use of a math function in the WHERE clause
+and sin(cfloat) >= -1.0
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@alltypesorc
+#### A masked pattern was here ####
+-200.0	-200.0	-200	-200	0.8199077823142826	NULL	NULL	NULL	NULL	NULL	NULL	30.57531565116074	NULL	NULL	NULL	NULL	NULL	NULL	NULL	39998.48747140321	1011111010111000011011101011000	2D3230302E30	5F5C3758	200.0	36	NULL	0.8732972972139946	NaN	0.4871876750070059	NaN	-1.5657963684609384	-11459.155902616465	-3.490658503988659	-200.0	1599879000	200.0	-1.0	1.0	NULL
+15601.0	15601.0	15601	15601	0.38656833237681376	15601.00000000001	9.65509029374725	NULL	4.193152436852078	13.929350886124324	NULL	NULL	NULL	13.929350886124324	NULL	13.929350886124324	194.02681610877246	194.02681610877246	124.90396310766124	NULL	1111111111111111111111111111111110010001001101101010100000011000	31353630312E30	FFFFFFFF9136A818	15601.0	38	NULL	-0.14856570831397706	NaN	0.9889025383288114	NaN	1.5707322283397571	893871.4561835973	272.2888166036353	15601.0	-1858689000	-15601.0	1.0	-1.0	-0.9740573096878733
+15601.0	15601.0	15601	15601	0.41161398527282966	15601.00000000001	9.65509029374725	NULL	4.193152436852078	13.929350886124324	NULL	29.18993673432575	NULL	13.929350886124324	NULL	13.929350886124324	194.02681610877246	194.02681610877246	124.90396310766124	24747.04022706554	100100100000001011101000000000	31353630312E30	2480BA00	15601.0	5	NULL	-0.14856570831397706	NaN	0.9889025383288114	NaN	1.5707322283397571	893871.4561835973	272.2888166036353	15601.0	612416000	-15601.0	1.0	1.0	-0.9740573096878733
+15601.0	15601.0	15601	15601	0.37807863784568585	15601.00000000001	9.65509029374725	3.871201010907891	4.193152436852078	13.929350886124324	5.584962500721157	NULL	NULL	13.929350886124324	5.584962500721157	13.929350886124324	194.02681610877246	194.02681610877246	124.90396310766124	NULL	1111111111111111111111111111111111010000100101111100000100011000	31353630312E30	FFFFFFFFD097C118	15601.0	48	NULL	-0.14856570831397706	NaN	0.9889025383288114	NaN	1.5707322283397571	893871.4561835973	272.2888166036353	15601.0	-795361000	-15601.0	1.0	-1.0	-0.9740573096878733
+NULL	NULL	NULL	NULL	0.3336458983920575	NULL	NULL	2.0794415416798357	NULL	NULL	3.0	29.693388204506274	29.58473549442715	NULL	3.0	NULL	NULL	NULL	NULL	29464.580431426475	110011101111110001011111011100	NULL	33BF17DC	NULL	8	1	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	868161500	NULL	NULL	1.0	NULL
+NULL	NULL	NULL	NULL	0.8681331660942196	NULL	NULL	2.0794415416798357	NULL	NULL	3.0	NULL	29.730832334348488	NULL	3.0	NULL	NULL	NULL	NULL	NULL	1111111111111111111111111111111110010000111111111000101010111000	NULL	FFFFFFFF90FF8AB8	NULL	8	0	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	-1862301000	NULL	NULL	-1.0	NULL
+-7196.0	-7196.0	-7196	-7196	0.03951015606275099	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	1111111111111111111111111111111110100000010101110101001001110000	2D373139362E30	FFFFFFFFA0575270	7196.0	59	NULL	-0.9834787875028149	NaN	-0.18102340879563897	NaN	-1.5706573607035177	-412300.4293761404	-125.59389297351194	-7196.0	-1604890000	7196.0	-1.0	-1.0	NULL
+-7196.0	-7196.0	-7196	-7196	0.9209252022050654	NULL	NULL	NULL	NULL	NULL	NULL	30.52255693577237	NULL	NULL	NULL	NULL	NULL	NULL	NULL	39273.76987252433	1011011111011111001100101001000	2D373139362E30	5BEF9948	7196.0	21	NULL	-0.9834787875028149	NaN	-0.18102340879563897	NaN	-1.5706573607035177	-412300.4293761404	-125.59389297351194	-7196.0	1542429000	7196.0	-1.0	1.0	NULL
+-7196.0	-7196.0	-7196	-7196	0.4533660450429132	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	1111111111111111111111111111111110100011011110110101000010110100	2D373139362E30	FFFFFFFFA37B50B4	7196.0	14	NULL	-0.9834787875028149	NaN	-0.18102340879563897	NaN	-1.5706573607035177	-412300.4293761404	-125.59389297351194	-7196.0	-1552199500	7196.0	-1.0	-1.0	NULL
+-7196.0	-7196.0	-7196	-7196	0.14567136069921982	NULL	NULL	4.07753744390572	NULL	NULL	5.882643049361842	NULL	NULL	NULL	5.882643049361842	NULL	NULL	NULL	NULL	NULL	1111111111111111111111111111111110111100001011110011111001111100	2D373139362E30	FFFFFFFFBC2F3E7C	7196.0	59	NULL	-0.9834787875028149	NaN	-0.18102340879563897	NaN	-1.5706573607035177	-412300.4293761404	-125.59389297351194	-7196.0	-1137754500	7196.0	-1.0	-1.0	NULL
+-7196.0	-7196.0	-7196	-7196	0.5264452612398715	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	1111111111111111111111111111111110010001101110110101111010110100	2D373139362E30	FFFFFFFF91BB5EB4	7196.0	8	NULL	-0.9834787875028149	NaN	-0.18102340879563897	NaN	-1.5706573607035177	-412300.4293761404	-125.59389297351194	-7196.0	-1849991500	7196.0	-1.0	-1.0	NULL
+-7196.0	-7196.0	-7196	-7196	0.17837094616515647	NULL	NULL	1.6094379124341003	NULL	NULL	2.321928094887362	NULL	NULL	NULL	2.321928094887362	NULL	NULL	NULL	NULL	NULL	1111111111111111111111111111111111000011011101110000111100110100	2D373139362E30	FFFFFFFFC3770F34	7196.0	5	NULL	-0.9834787875028149	NaN	-0.18102340879563897	NaN	-1.5706573607035177	-412300.4293761404	-125.59389297351194	-7196.0	-1015607500	7196.0	-1.0	-1.0	NULL
+-7196.0	-7196.0	-7196	-7196	0.5456857574763374	NULL	NULL	NULL	NULL	NULL	NULL	29.62699001935971	NULL	NULL	NULL	NULL	NULL	NULL	NULL	28794.287627930647	110001011010110011101011011000	2D373139362E30	316B3AD8	7196.0	24	NULL	-0.9834787875028149	NaN	-0.18102340879563897	NaN	-1.5706573607035177	-412300.4293761404	-125.59389297351194	-7196.0	829111000	7196.0	-1.0	1.0	NULL
+NULL	NULL	NULL	NULL	0.282703740641956	NULL	NULL	2.3978952727983707	NULL	NULL	3.4594316186372978	30.19990821555368	NULL	NULL	3.4594316186372978	NULL	NULL	NULL	NULL	35118.75567271711	1001001100000110001001110011000	NULL	49831398	NULL	11	1	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	1233327000	NULL	NULL	1.0	NULL

Added: hive/trunk/ql/src/test/results/clientpositive/vectorized_string_funcs.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientpositive/vectorized_string_funcs.q.out?rev=1533106&view=auto
==============================================================================
--- hive/trunk/ql/src/test/results/clientpositive/vectorized_string_funcs.q.out (added)
+++ hive/trunk/ql/src/test/results/clientpositive/vectorized_string_funcs.q.out Thu Oct 17 15:12:09 2013
@@ -0,0 +1,169 @@
+PREHOOK: query: -- Test string functions in vectorized mode to verify end-to-end functionality.
+
+explain 
+select 
+   substr(cstring1, 1, 2)
+  ,substr(cstring1, 2)
+  ,lower(cstring1)
+  ,upper(cstring1)
+  ,ucase(cstring1)
+  ,length(cstring1)
+  ,trim(cstring1)
+  ,ltrim(cstring1)
+  ,rtrim(cstring1)
+  ,concat(cstring1, cstring2)
+  ,concat('>', cstring1)
+  ,concat(cstring1, '<')
+  ,concat(substr(cstring1, 1, 2), substr(cstring2, 1, 2))
+from alltypesorc
+-- Limit the number of rows of output to a reasonable amount.
+where cbigint % 237 = 0
+-- Test function use in the WHERE clause.
+and length(substr(cstring1, 1, 2)) <= 2
+and cstring1 like '%'
+PREHOOK: type: QUERY
+POSTHOOK: query: -- Test string functions in vectorized mode to verify end-to-end functionality.
+
+explain 
+select 
+   substr(cstring1, 1, 2)
+  ,substr(cstring1, 2)
+  ,lower(cstring1)
+  ,upper(cstring1)
+  ,ucase(cstring1)
+  ,length(cstring1)
+  ,trim(cstring1)
+  ,ltrim(cstring1)
+  ,rtrim(cstring1)
+  ,concat(cstring1, cstring2)
+  ,concat('>', cstring1)
+  ,concat(cstring1, '<')
+  ,concat(substr(cstring1, 1, 2), substr(cstring2, 1, 2))
+from alltypesorc
+-- Limit the number of rows of output to a reasonable amount.
+where cbigint % 237 = 0
+-- Test function use in the WHERE clause.
+and length(substr(cstring1, 1, 2)) <= 2
+and cstring1 like '%'
+POSTHOOK: type: QUERY
+ABSTRACT SYNTAX TREE:
+  (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME alltypesorc))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_FUNCTION substr (TOK_TABLE_OR_COL cstring1) 1 2)) (TOK_SELEXPR (TOK_FUNCTION substr (TOK_TABLE_OR_COL cstring1) 2)) (TOK_SELEXPR (TOK_FUNCTION lower (TOK_TABLE_OR_COL cstring1))) (TOK_SELEXPR (TOK_FUNCTION upper (TOK_TABLE_OR_COL cstring1))) (TOK_SELEXPR (TOK_FUNCTION ucase (TOK_TABLE_OR_COL cstring1))) (TOK_SELEXPR (TOK_FUNCTION length (TOK_TABLE_OR_COL cstring1))) (TOK_SELEXPR (TOK_FUNCTION trim (TOK_TABLE_OR_COL cstring1))) (TOK_SELEXPR (TOK_FUNCTION ltrim (TOK_TABLE_OR_COL cstring1))) (TOK_SELEXPR (TOK_FUNCTION rtrim (TOK_TABLE_OR_COL cstring1))) (TOK_SELEXPR (TOK_FUNCTION concat (TOK_TABLE_OR_COL cstring1) (TOK_TABLE_OR_COL cstring2))) (TOK_SELEXPR (TOK_FUNCTION concat '>' (TOK_TABLE_OR_COL cstring1))) (TOK_SELEXPR (TOK_FUNCTION concat (TOK_TABLE_OR_COL cstring1) '<')) (TOK_SELEXPR (TOK_FUNCTION concat (TOK_FUNCTION substr (TOK_TABLE_
 OR_COL cstring1) 1 2) (TOK_FUNCTION substr (TOK_TABLE_OR_COL cstring2) 1 2)))) (TOK_WHERE (and (and (= (% (TOK_TABLE_OR_COL cbigint) 237) 0) (<= (TOK_FUNCTION length (TOK_FUNCTION substr (TOK_TABLE_OR_COL cstring1) 1 2)) 2)) (like (TOK_TABLE_OR_COL cstring1) '%')))))
+
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 is a root stage
+
+STAGE PLANS:
+  Stage: Stage-1
+    Map Reduce
+      Alias -> Map Operator Tree:
+        alltypesorc 
+          TableScan
+            alias: alltypesorc
+            Filter Operator
+              predicate:
+                  expr: ((((cbigint % 237) = 0) and (length(substr(cstring1, 1, 2)) <= 2)) and (cstring1 like '%'))
+                  type: boolean
+              Vectorized execution: true
+              Select Operator
+                expressions:
+                      expr: substr(cstring1, 1, 2)
+                      type: string
+                      expr: substr(cstring1, 2)
+                      type: string
+                      expr: lower(cstring1)
+                      type: string
+                      expr: upper(cstring1)
+                      type: string
+                      expr: upper(cstring1)
+                      type: string
+                      expr: length(cstring1)
+                      type: int
+                      expr: trim(cstring1)
+                      type: string
+                      expr: ltrim(cstring1)
+                      type: string
+                      expr: rtrim(cstring1)
+                      type: string
+                      expr: concat(cstring1, cstring2)
+                      type: string
+                      expr: concat('>', cstring1)
+                      type: string
+                      expr: concat(cstring1, '<')
+                      type: string
+                      expr: concat(substr(cstring1, 1, 2), substr(cstring2, 1, 2))
+                      type: string
+                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
+                Vectorized execution: true
+                File Output Operator
+                  compressed: false
+                  GlobalTableId: 0
+                  table:
+                      input format: org.apache.hadoop.mapred.TextInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                  Vectorized execution: true
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+
+
+PREHOOK: query: select 
+   substr(cstring1, 1, 2)
+  ,substr(cstring1, 2)
+  ,lower(cstring1)
+  ,upper(cstring1)
+  ,ucase(cstring1)
+  ,length(cstring1)
+  ,trim(cstring1)
+  ,ltrim(cstring1)
+  ,rtrim(cstring1)
+  ,concat(cstring1, cstring2)
+  ,concat('>', cstring1)
+  ,concat(cstring1, '<')
+  ,concat(substr(cstring1, 1, 2), substr(cstring2, 1, 2))
+from alltypesorc
+-- Limit the number of rows of output to a reasonable amount.
+where cbigint % 237 = 0
+-- Test function use in the WHERE clause.
+and length(substr(cstring1, 1, 2)) <= 2
+and cstring1 like '%'
+PREHOOK: type: QUERY
+PREHOOK: Input: default@alltypesorc
+#### A masked pattern was here ####
+POSTHOOK: query: select 
+   substr(cstring1, 1, 2)
+  ,substr(cstring1, 2)
+  ,lower(cstring1)
+  ,upper(cstring1)
+  ,ucase(cstring1)
+  ,length(cstring1)
+  ,trim(cstring1)
+  ,ltrim(cstring1)
+  ,rtrim(cstring1)
+  ,concat(cstring1, cstring2)
+  ,concat('>', cstring1)
+  ,concat(cstring1, '<')
+  ,concat(substr(cstring1, 1, 2), substr(cstring2, 1, 2))
+from alltypesorc
+-- Limit the number of rows of output to a reasonable amount.
+where cbigint % 237 = 0
+-- Test function use in the WHERE clause.
+and length(substr(cstring1, 1, 2)) <= 2
+and cstring1 like '%'
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@alltypesorc
+#### A masked pattern was here ####
+Vi	iqXS6s88N1yr14lj7I	viqxs6s88n1yr14lj7i	VIQXS6S88N1YR14LJ7I	VIQXS6S88N1YR14LJ7I	19	ViqXS6s88N1yr14lj7I	ViqXS6s88N1yr14lj7I	ViqXS6s88N1yr14lj7I	ViqXS6s88N1yr14lj7ITh638b67kn8o	>ViqXS6s88N1yr14lj7I	ViqXS6s88N1yr14lj7I<	ViTh
+R4	4e7Gf	r4e7gf	R4E7GF	R4E7GF	6	R4e7Gf	R4e7Gf	R4e7Gf	R4e7GfPTBh56R3LS7L13sB4	>R4e7Gf	R4e7Gf<	R4PT
+3g	gubGh4J18TV	3gubgh4j18tv	3GUBGH4J18TV	3GUBGH4J18TV	12	3gubGh4J18TV	3gubGh4J18TV	3gubGh4J18TV	3gubGh4J18TVpJucOe4dN4R5XURJW8	>3gubGh4J18TV	3gubGh4J18TV<	3gpJ
+EP	PCRx8ObNv51rOF	epcrx8obnv51rof	EPCRX8OBNV51ROF	EPCRX8OBNV51ROF	15	EPCRx8ObNv51rOF	EPCRx8ObNv51rOF	EPCRx8ObNv51rOF	EPCRx8ObNv51rOFysaU2Xm11f715L0I35rut2	>EPCRx8ObNv51rOF	EPCRx8ObNv51rOF<	EPys
+8e	eiti74gc5m01xyMKSjUIx	8eiti74gc5m01xymksjuix	8EITI74GC5M01XYMKSJUIX	8EITI74GC5M01XYMKSJUIX	22	8eiti74gc5m01xyMKSjUIx	8eiti74gc5m01xyMKSjUIx	8eiti74gc5m01xyMKSjUIx	8eiti74gc5m01xyMKSjUIxI8x87Fm1J4hE8g4CWNo	>8eiti74gc5m01xyMKSjUIx	8eiti74gc5m01xyMKSjUIx<	8eI8
+m0	0hbv1516qk8	m0hbv1516qk8	M0HBV1516QK8	M0HBV1516QK8	12	m0hbv1516qk8	m0hbv1516qk8	m0hbv1516qk8	m0hbv1516qk8N8i3sxF54C4x5h0	>m0hbv1516qk8	m0hbv1516qk8<	m0N8
+uT	T5e2	ut5e2	UT5E2	UT5E2	5	uT5e2	uT5e2	uT5e2	uT5e2SJp57VKYsDtA2r1Xb2H	>uT5e2	uT5e2<	uTSJ
+l3	35W8012cM77E227Ts	l35w8012cm77e227ts	L35W8012CM77E227TS	L35W8012CM77E227TS	18	l35W8012cM77E227Ts	l35W8012cM77E227Ts	l35W8012cM77E227Ts	l35W8012cM77E227TsMH38bE	>l35W8012cM77E227Ts	l35W8012cM77E227Ts<	l3MH
+o1	1uPH5EflET5ts1RjSB74	o1uph5eflet5ts1rjsb74	O1UPH5EFLET5TS1RJSB74	O1UPH5EFLET5TS1RJSB74	21	o1uPH5EflET5ts1RjSB74	o1uPH5EflET5ts1RjSB74	o1uPH5EflET5ts1RjSB74	o1uPH5EflET5ts1RjSB74a1U3DRA788kW7I0UTF203	>o1uPH5EflET5ts1RjSB74	o1uPH5EflET5ts1RjSB74<	o1a1
+Ix	x8dXlDbC3S44L1FQJqpwa	ix8dxldbc3s44l1fqjqpwa	IX8DXLDBC3S44L1FQJQPWA	IX8DXLDBC3S44L1FQJQPWA	22	Ix8dXlDbC3S44L1FQJqpwa	Ix8dXlDbC3S44L1FQJqpwa	Ix8dXlDbC3S44L1FQJqpwa	Ix8dXlDbC3S44L1FQJqpwa8wQR4X28CiccBVXGqPL7	>Ix8dXlDbC3S44L1FQJqpwa	Ix8dXlDbC3S44L1FQJqpwa<	Ix8w
+OT	Tn0Dj2HiBi05Baq1Xt	otn0dj2hibi05baq1xt	OTN0DJ2HIBI05BAQ1XT	OTN0DJ2HIBI05BAQ1XT	19	OTn0Dj2HiBi05Baq1Xt	OTn0Dj2HiBi05Baq1Xt	OTn0Dj2HiBi05Baq1Xt	OTn0Dj2HiBi05Baq1XtAoQ21J1lQ27kYSmfA	>OTn0Dj2HiBi05Baq1Xt	OTn0Dj2HiBi05Baq1Xt<	OTAo
+a0	0P3sn1ihxJCsTLDb	a0p3sn1ihxjcstldb	A0P3SN1IHXJCSTLDB	A0P3SN1IHXJCSTLDB	17	a0P3sn1ihxJCsTLDb	a0P3sn1ihxJCsTLDb	a0P3sn1ihxJCsTLDb	a0P3sn1ihxJCsTLDbfT4Jlw38k8kmd6Dt1wv	>a0P3sn1ihxJCsTLDb	a0P3sn1ihxJCsTLDb<	a0fT