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/10 00:58:40 UTC

svn commit: r1530824 - in /hive/trunk: ant/src/org/apache/hadoop/hive/ant/ ql/src/gen/vectorization/ExpressionTemplates/ ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/ ql/src/java/org/a...

Author: hashutosh
Date: Wed Oct  9 22:58:40 2013
New Revision: 1530824

URL: http://svn.apache.org/r1530824
Log:
HIVE-4898 : make vectorized math functions work end-to-end (update VectorizationContext.java) (Eric Hanson via Ashutosh Chauhan)

Added:
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncLogWithBaseLongToDouble.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncPowerLongToDouble.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/ISetDoubleArg.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/ISetLongArg.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/MathFuncLongToDouble.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/RoundWithNumDigitsDoubleToDouble.java
Removed:
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/RoundDoubleToDoubleWithNumDigits.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/StringConv.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/StringUnhex.java
Modified:
    hive/trunk/ant/src/org/apache/hadoop/hive/ant/GenVectorCode.java
    hive/trunk/ql/src/gen/vectorization/ExpressionTemplates/ColumnUnaryFunc.txt
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncBin.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncHex.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncLogWithBaseDoubleToDouble.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncPowerDoubleToDouble.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncRand.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/MathExpr.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/MathFuncDoubleToDouble.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/ant/src/org/apache/hadoop/hive/ant/GenVectorCode.java
URL: http://svn.apache.org/viewvc/hive/trunk/ant/src/org/apache/hadoop/hive/ant/GenVectorCode.java?rev=1530824&r1=1530823&r2=1530824&view=diff
==============================================================================
--- hive/trunk/ant/src/org/apache/hadoop/hive/ant/GenVectorCode.java (original)
+++ hive/trunk/ant/src/org/apache/hadoop/hive/ant/GenVectorCode.java Wed Oct  9 22:58:40 2013
@@ -303,45 +303,64 @@ public class GenVectorCode extends Task 
       {"ColumnCompareColumn", "GreaterEqual", "double", "long", ">="},
 
       // template, <ClassNamePrefix>, <ReturnType>, <OperandType>, <FuncName>, <OperandCast>,
-      //   <ResultCast>
-      {"ColumnUnaryFunc", "FuncRound", "double", "double", "MathExpr.round", "", ""},
+      //   <ResultCast>, <Cleanup>
+      {"ColumnUnaryFunc", "FuncRound", "double", "double", "MathExpr.round", "", "", ""},
       // round(longCol) returns a long and is a no-op. So it will not be implemented here.
       // round(Col, N) is a special case and will be implemented separately from this template
-      {"ColumnUnaryFunc", "FuncFloor", "long", "double", "Math.floor", "", "(long)"},
-      // Note: floor(long) is a no-op so code generation should remove it or use
-      // an IdentityExpression
-      {"ColumnUnaryFunc", "FuncCeil", "long", "double", "Math.ceil", "", "(long)"},
-      // Similarly, ceil(long) is a no-op, so not generating code for it here
-      {"ColumnUnaryFunc", "FuncExp", "double", "double", "Math.exp", "", ""},
-      {"ColumnUnaryFunc", "FuncLn", "double", "double", "Math.log", "", ""},
-      {"ColumnUnaryFunc", "FuncLn", "double", "long", "Math.log", "(double)", ""},
-      {"ColumnUnaryFunc", "FuncLog10", "double", "double", "Math.log10", "", ""},
-      {"ColumnUnaryFunc", "FuncLog10", "double", "long", "Math.log10", "(double)", ""},
+      {"ColumnUnaryFunc", "FuncFloor", "long", "double", "Math.floor", "", "(long)", ""},
+      // Floor on an integer argument is a noop, but it is less code to handle it this way.
+      {"ColumnUnaryFunc", "FuncFloor", "long", "long", "Math.floor", "", "(long)", ""},
+      {"ColumnUnaryFunc", "FuncCeil", "long", "double", "Math.ceil", "", "(long)", ""},
+      // Ceil on an integer argument is a noop, but it is less code to handle it this way.
+      {"ColumnUnaryFunc", "FuncCeil", "long", "long", "Math.ceil", "", "(long)", ""},
+      {"ColumnUnaryFunc", "FuncExp", "double", "double", "Math.exp", "", "", ""},
+      {"ColumnUnaryFunc", "FuncExp", "double", "long", "Math.exp", "(double)", "", ""},
+      {"ColumnUnaryFunc", "FuncLn", "double", "double", "Math.log", "", "",
+        "MathExpr.NaNToNull(outputColVector, sel, batch.selectedInUse, n);"},
+      {"ColumnUnaryFunc", "FuncLn", "double", "long", "Math.log", "(double)", "",
+        "MathExpr.NaNToNull(outputColVector, sel, batch.selectedInUse, n);"},
+      {"ColumnUnaryFunc", "FuncLog10", "double", "double", "Math.log10", "", "",
+        "MathExpr.NaNToNull(outputColVector, sel, batch.selectedInUse, n);"},
+      {"ColumnUnaryFunc", "FuncLog10", "double", "long", "Math.log10", "(double)", "",
+        "MathExpr.NaNToNull(outputColVector, sel, batch.selectedInUse, n);"},
       // The MathExpr class contains helper functions for cases when existing library
       // routines can't be used directly.
-      {"ColumnUnaryFunc", "FuncLog2", "double", "double", "MathExpr.log2", "", ""},
-      {"ColumnUnaryFunc", "FuncLog2", "double", "long", "MathExpr.log2", "(double)", ""},
+      {"ColumnUnaryFunc", "FuncLog2", "double", "double", "MathExpr.log2", "", "",
+        "MathExpr.NaNToNull(outputColVector, sel, batch.selectedInUse, n);"},
+      {"ColumnUnaryFunc", "FuncLog2", "double", "long", "MathExpr.log2", "(double)", "",
+        "MathExpr.NaNToNull(outputColVector, sel, batch.selectedInUse, n);"},
       // Log(base, Col) is a special case and will be implemented separately from this template
       // Pow(col, P) and Power(col, P) are special cases implemented separately from this template
-      {"ColumnUnaryFunc", "FuncSqrt", "double", "double", "Math.sqrt", "", ""},
-      {"ColumnUnaryFunc", "FuncSqrt", "double", "long", "Math.sqrt", "(double)", ""},
-      {"ColumnUnaryFunc", "FuncAbs", "double", "double", "Math.abs", "", ""},
-      {"ColumnUnaryFunc", "FuncAbs", "long", "long", "MathExpr.abs", "", ""},
-      {"ColumnUnaryFunc", "FuncSin", "double", "double", "Math.sin", "", ""},
-      {"ColumnUnaryFunc", "FuncASin", "double", "double", "Math.asin", "", ""},
-      {"ColumnUnaryFunc", "FuncCos", "double", "double", "Math.cos", "", ""},
-      {"ColumnUnaryFunc", "FuncACos", "double", "double", "Math.acos", "", ""},
-      {"ColumnUnaryFunc", "FuncTan", "double", "double", "Math.tan", "", ""},
-      {"ColumnUnaryFunc", "FuncATan", "double", "double", "Math.atan", "", ""},
-      {"ColumnUnaryFunc", "FuncDegrees", "double", "double", "Math.toDegrees", "", ""},
-      {"ColumnUnaryFunc", "FuncRadians", "double", "double", "Math.toRadians", "", ""},
-      {"ColumnUnaryFunc", "FuncSign", "double", "double", "MathExpr.sign", "", ""},
-      {"ColumnUnaryFunc", "FuncSign", "double", "long", "MathExpr.sign", "", ""},
+      {"ColumnUnaryFunc", "FuncSqrt", "double", "double", "Math.sqrt", "", "",
+        "MathExpr.NaNToNull(outputColVector, sel, batch.selectedInUse, n);"},
+      {"ColumnUnaryFunc", "FuncSqrt", "double", "long", "Math.sqrt", "(double)", "",
+        "MathExpr.NaNToNull(outputColVector, sel, batch.selectedInUse, n);"},
+      {"ColumnUnaryFunc", "FuncAbs", "double", "double", "Math.abs", "", "", ""},
+      {"ColumnUnaryFunc", "FuncAbs", "long", "long", "MathExpr.abs", "", "", ""},
+      {"ColumnUnaryFunc", "FuncSin", "double", "double", "Math.sin", "", "", ""},
+      {"ColumnUnaryFunc", "FuncSin", "double", "long", "Math.sin", "(double)", "", ""},
+      {"ColumnUnaryFunc", "FuncASin", "double", "double", "Math.asin", "", "", ""},
+      {"ColumnUnaryFunc", "FuncASin", "double", "long", "Math.asin", "(double)", "", ""},
+      {"ColumnUnaryFunc", "FuncCos", "double", "double", "Math.cos", "", "", ""},
+      {"ColumnUnaryFunc", "FuncCos", "double", "long", "Math.cos", "(double)", "", ""},
+      {"ColumnUnaryFunc", "FuncACos", "double", "double", "Math.acos", "", "", ""},
+      {"ColumnUnaryFunc", "FuncACos", "double", "long", "Math.acos", "(double)", "", ""},
+      {"ColumnUnaryFunc", "FuncTan", "double", "double", "Math.tan", "", "", ""},
+      {"ColumnUnaryFunc", "FuncTan", "double", "long", "Math.tan", "(double)", "", ""},
+      {"ColumnUnaryFunc", "FuncATan", "double", "double", "Math.atan", "", "", ""},
+      {"ColumnUnaryFunc", "FuncATan", "double", "long", "Math.atan", "(double)", "", ""},
+      {"ColumnUnaryFunc", "FuncDegrees", "double", "double", "Math.toDegrees", "", "", ""},
+      {"ColumnUnaryFunc", "FuncDegrees", "double", "long", "Math.toDegrees", "(double)", "", ""},
+      {"ColumnUnaryFunc", "FuncRadians", "double", "double", "Math.toRadians", "", "", ""},
+      {"ColumnUnaryFunc", "FuncRadians", "double", "long", "Math.toRadians", "(double)", "", ""},
+      {"ColumnUnaryFunc", "FuncSign", "double", "double", "MathExpr.sign", "", "", ""},
+      {"ColumnUnaryFunc", "FuncSign", "double", "long", "MathExpr.sign", "(double)", "", ""},
 
 
         {"ColumnUnaryMinus", "long"},
         {"ColumnUnaryMinus", "double"},
 
+
       // template, <ClassName>, <ValueType>, <OperatorSymbol>, <DescriptionName>, <DescriptionValue>
       {"VectorUDAFMinMax", "VectorUDAFMinLong", "long", "<", "min",
           "_FUNC_(expr) - Returns the minimum value of expr (vectorized, type: long)"},
@@ -722,6 +741,7 @@ public class GenVectorCode extends Task 
     String funcName = tdesc[4];
     String operandCast = tdesc[5];
     String resultCast = tdesc[6];
+    String cleanup = tdesc[7];
     // Expand, and write result
     templateString = templateString.replaceAll("<ClassName>", className);
     templateString = templateString.replaceAll("<InputColumnVectorType>", inputColumnVectorType);
@@ -731,6 +751,7 @@ public class GenVectorCode extends Task 
     templateString = templateString.replaceAll("<FuncName>", funcName);
     templateString = templateString.replaceAll("<OperandCast>", operandCast);
     templateString = templateString.replaceAll("<ResultCast>", resultCast);
+    templateString = templateString.replaceAll("<Cleanup>", cleanup);
     writeFile(outputFile, templateString);
   }
 

Modified: hive/trunk/ql/src/gen/vectorization/ExpressionTemplates/ColumnUnaryFunc.txt
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/gen/vectorization/ExpressionTemplates/ColumnUnaryFunc.txt?rev=1530824&r1=1530823&r2=1530824&view=diff
==============================================================================
--- hive/trunk/ql/src/gen/vectorization/ExpressionTemplates/ColumnUnaryFunc.txt (original)
+++ hive/trunk/ql/src/gen/vectorization/ExpressionTemplates/ColumnUnaryFunc.txt Wed Oct  9 22:58:40 2013
@@ -95,6 +95,7 @@ public class <ClassName> extends VectorE
       }
       outputColVector.isRepeating = false;
     }
+    <Cleanup>
   }
 
   @Override

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=1530824&r1=1530823&r2=1530824&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 Wed Oct  9 22:58:40 2013
@@ -41,6 +41,9 @@ import org.apache.hadoop.hive.ql.exec.ve
 import org.apache.hadoop.hive.ql.exec.vector.expressions.FilterStringColLikeStringScalar;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.FilterStringColRegExpStringScalar;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.IdentityExpression;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.FuncRand;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.ISetDoubleArg;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.ISetLongArg;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.SelectColumnIsNotNull;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.SelectColumnIsNull;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.SelectColumnIsTrue;
@@ -102,6 +105,21 @@ import org.apache.hadoop.hive.ql.udf.UDF
 import org.apache.hadoop.hive.ql.udf.UDFTrim;
 import org.apache.hadoop.hive.ql.udf.UDFWeekOfYear;
 import org.apache.hadoop.hive.ql.udf.UDFYear;
+import org.apache.hadoop.hive.ql.udf.UDFAcos;
+import org.apache.hadoop.hive.ql.udf.UDFAsin;
+import org.apache.hadoop.hive.ql.udf.UDFAtan;
+import org.apache.hadoop.hive.ql.udf.UDFBin;
+import org.apache.hadoop.hive.ql.udf.UDFCeil;
+import org.apache.hadoop.hive.ql.udf.UDFConv;
+import org.apache.hadoop.hive.ql.udf.UDFCos;
+import org.apache.hadoop.hive.ql.udf.UDFDegrees;
+import org.apache.hadoop.hive.ql.udf.UDFExp;
+import org.apache.hadoop.hive.ql.udf.UDFFloor;
+import org.apache.hadoop.hive.ql.udf.UDFHex;
+import org.apache.hadoop.hive.ql.udf.UDFLn;
+import org.apache.hadoop.hive.ql.udf.UDFLog;
+import org.apache.hadoop.hive.ql.udf.UDFLog10;
+import org.apache.hadoop.hive.ql.udf.UDFLog2;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDF;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDFBridge;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDFConcat;
@@ -119,8 +137,21 @@ import org.apache.hadoop.hive.ql.udf.gen
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPOr;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDFToUnixTimeStamp;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDFUpper;
+import org.apache.hadoop.hive.ql.udf.UDFPower;
+import org.apache.hadoop.hive.ql.udf.UDFRadians;
+import org.apache.hadoop.hive.ql.udf.UDFRand;
+import org.apache.hadoop.hive.ql.udf.UDFRound;
+import org.apache.hadoop.hive.ql.udf.UDFSign;
+import org.apache.hadoop.hive.ql.udf.UDFSin;
+import org.apache.hadoop.hive.ql.udf.UDFSqrt;
+import org.apache.hadoop.hive.ql.udf.UDFTan;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDFAbs;
+
+import org.apache.hadoop.hive.serde2.io.DoubleWritable;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils;
+import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.Text;
 
 /**
@@ -141,6 +172,14 @@ public class VectorizationContext {
   //Map column number to type
   private final OutputColumnManager ocm;
 
+  // Package where custom (hand-built) vector expression classes are located.
+  private static final String CUSTOM_EXPR_PACKAGE =
+      "org.apache.hadoop.hive.ql.exec.vector.expressions";
+
+  // Package where vector expression packages generated from templates are located.
+  private static final String GENERATED_EXPR_PACKAGE =
+      "org.apache.hadoop.hive.ql.exec.vector.expressions.gen";
+
   public VectorizationContext(Map<String, Integer> columnMap,
       int initialOutputCol) {
     this.columnMap = columnMap;
@@ -254,7 +293,7 @@ public class VectorizationContext {
       ve = getVectorExpression((ExprNodeColumnDesc) exprDesc);
     } else if (exprDesc instanceof ExprNodeGenericFuncDesc) {
       ExprNodeGenericFuncDesc expr = (ExprNodeGenericFuncDesc) exprDesc;
-      if (isCustomUDF(expr)) {
+      if (isCustomUDF(expr) || isLegacyPathUDF(expr)) {
         ve = getCustomUDFExpression(expr);
       } else {
         ve = getVectorExpression(expr.getGenericUDF(),
@@ -269,6 +308,26 @@ public class VectorizationContext {
     return ve;
   }
 
+  /* Return true if this is one of a small set of functions for which
+   * it is significantly easier to use the old code path in vectorized
+   * mode instead of implementing a new, optimized VectorExpression.
+   *
+   * Depending on performance requirements and frequency of use, these
+   * may be implemented in the future with an optimized VectorExpression.
+   */
+  public static boolean isLegacyPathUDF(ExprNodeGenericFuncDesc expr) {
+    GenericUDF gudf = expr.getGenericUDF();
+    if (gudf instanceof GenericUDFBridge) {
+      GenericUDFBridge bridge = (GenericUDFBridge) gudf;
+      Class<? extends UDF> udfClass = bridge.getUdfClass();
+      if (udfClass.equals(UDFHex.class)
+          || udfClass.equals(UDFConv.class)) {
+        return true;
+      }
+    }
+    return false;
+  }
+
   // Return true if this is a custom UDF or custom GenericUDF.
   // This is for use only in the planner. It will fail in a task.
   public static boolean isCustomUDF(ExprNodeGenericFuncDesc expr) {
@@ -388,6 +447,49 @@ public class VectorizationContext {
     return expr;
   }
 
+  /* For functions that take one argument, and can be translated using a vector
+   * expression class of the form
+   *   <packagePrefix>.<classPrefix><argumentType>To<resultType>
+   * The argumentType is inferred from the input expression.
+   */
+  private VectorExpression getUnaryFunctionExpression(
+      String classPrefix,
+      String resultType,
+      List<ExprNodeDesc> childExprList,
+      String packagePrefix)
+      throws HiveException {
+    ExprNodeDesc childExpr = childExprList.get(0);
+    int inputCol;
+    String colType;
+    VectorExpression v1 = null;
+    if (childExpr instanceof ExprNodeGenericFuncDesc) {
+      v1 = getVectorExpression(childExpr);
+      inputCol = v1.getOutputColumn();
+      colType = v1.getOutputType();
+    } else if (childExpr instanceof ExprNodeColumnDesc) {
+      ExprNodeColumnDesc colDesc = (ExprNodeColumnDesc) childExpr;
+      inputCol = getInputColumnIndex(colDesc.getColumn());
+      colType = colDesc.getTypeString();
+    } else {
+      throw new HiveException("Expression not supported: "+childExpr);
+    }
+    String funcInputColType = getNormalizedTypeName(colType);
+    int outputCol = ocm.allocateOutputColumn(resultType);
+    String className = packagePrefix + "."
+        + classPrefix + funcInputColType + "To" + resultType;
+    VectorExpression expr;
+    try {
+      expr = (VectorExpression) getConstructor(className).newInstance(inputCol, outputCol);
+    } catch (Exception ex) {
+      throw new HiveException(ex);
+    }
+    if (v1 != null) {
+      expr.setChildExpressions(new VectorExpression [] {v1});
+      ocm.freeOutputColumn(v1.getOutputColumn());
+    }
+    return expr;
+  }
+
   private VectorExpression getUnaryPlusExpression(List<ExprNodeDesc> childExprList)
       throws HiveException {
     ExprNodeDesc childExpr = childExprList.get(0);
@@ -446,11 +548,27 @@ public class VectorizationContext {
       return getUnaryStringExpression("StringUpper", "String", childExpr);
     } else if (udf instanceof GenericUDFConcat) {
       return getConcatExpression(childExpr);
+    } else if (udf instanceof GenericUDFAbs) {
+      return getUnaryAbsExpression(childExpr);
     }
 
     throw new HiveException("Udf: "+udf.getClass().getSimpleName()+", is not supported");
   }
 
+  private VectorExpression getUnaryAbsExpression(List<ExprNodeDesc> childExpr)
+      throws HiveException {
+    String argType = childExpr.get(0).getTypeString();
+    if (isIntFamily(argType)) {
+      return getUnaryFunctionExpression("FuncAbs", "Long", childExpr,
+          GENERATED_EXPR_PACKAGE);
+    } else if (isFloatFamily(argType)) {
+      return getUnaryFunctionExpression("FuncAbs", "Double", childExpr,
+          GENERATED_EXPR_PACKAGE);
+    }
+
+    throw new HiveException("Udf: Abs() not supported for argument type " + argType);
+  }
+
   private VectorExpression getVectorExpression(GenericUDFToUnixTimeStamp udf,
       List<ExprNodeDesc> childExpr) throws HiveException {
     ExprNodeDesc leftExpr = childExpr.get(0);
@@ -512,11 +630,194 @@ public class VectorizationContext {
       return getUnaryStringExpression("StringRTrim", "String", childExpr);
     } else if (cl.equals(UDFTrim.class)) {
       return getUnaryStringExpression("StringTrim", "String", childExpr);
+    } else if (cl.equals(UDFSin.class)) {
+      return getUnaryFunctionExpression("FuncSin", "Double", childExpr,
+          GENERATED_EXPR_PACKAGE);
+    } else if (cl.equals(UDFCos.class)) {
+      return getUnaryFunctionExpression("FuncCos", "Double", childExpr,
+          GENERATED_EXPR_PACKAGE);
+    } else if (cl.equals(UDFTan.class)) {
+      return getUnaryFunctionExpression("FuncTan", "Double", childExpr,
+          GENERATED_EXPR_PACKAGE);
+    } else if (cl.equals(UDFAsin.class)) {
+      return getUnaryFunctionExpression("FuncASin", "Double", childExpr,
+          GENERATED_EXPR_PACKAGE);
+    } else if (cl.equals(UDFAcos.class)) {
+      return getUnaryFunctionExpression("FuncACos", "Double", childExpr,
+          GENERATED_EXPR_PACKAGE);
+    } else if (cl.equals(UDFAtan.class)) {
+      return getUnaryFunctionExpression("FuncATan", "Double", childExpr,
+          GENERATED_EXPR_PACKAGE);
+    } else if (cl.equals(UDFFloor.class)) {
+      return getUnaryFunctionExpression("FuncFloor", "Long", childExpr,
+          GENERATED_EXPR_PACKAGE);
+    } else if (cl.equals(UDFCeil.class)) {
+      return getUnaryFunctionExpression("FuncCeil", "Long", childExpr,
+          GENERATED_EXPR_PACKAGE);
+    } else if (cl.equals(UDFDegrees.class)) {
+      return getUnaryFunctionExpression("FuncDegrees", "Double", childExpr,
+          GENERATED_EXPR_PACKAGE);
+    } else if (cl.equals(UDFRadians.class)) {
+      return getUnaryFunctionExpression("FuncRadians", "Double", childExpr,
+          GENERATED_EXPR_PACKAGE);
+    } else if (cl.equals(UDFLn.class)) {
+      return getUnaryFunctionExpression("FuncLn", "Double", childExpr,
+          GENERATED_EXPR_PACKAGE);
+    } else if (cl.equals(UDFLog2.class)) {
+      return getUnaryFunctionExpression("FuncLog2", "Double", childExpr,
+          GENERATED_EXPR_PACKAGE);
+    } else if (cl.equals(UDFLog10.class)) {
+      return getUnaryFunctionExpression("FuncLog10", "Double", childExpr,
+          GENERATED_EXPR_PACKAGE);
+    } else if (cl.equals(UDFSign.class)) {
+      return getUnaryFunctionExpression("FuncSign", "Double", childExpr,
+          GENERATED_EXPR_PACKAGE);
+    } else if (cl.equals(UDFSqrt.class)) {
+      return getUnaryFunctionExpression("FuncSqrt", "Double", childExpr,
+          GENERATED_EXPR_PACKAGE);
+    } else if (cl.equals(UDFExp.class)) {
+      return getUnaryFunctionExpression("FuncExp", "Double", childExpr,
+          GENERATED_EXPR_PACKAGE);
+    } else if (cl.equals(UDFLog.class)) {
+      return getLogWithBaseExpression(childExpr);
+    } else if (cl.equals(UDFPower.class)) {
+      return getPowerExpression(childExpr);
+    } else if (cl.equals(UDFRound.class)) {
+      return getRoundExpression(childExpr);
+    } else if (cl.equals(UDFRand.class)) {
+      return getRandExpression(childExpr);
+    } else if (cl.equals(UDFBin.class)) {
+      return getUnaryStringExpression("FuncBin", "String", childExpr);
     }
 
     throw new HiveException("Udf: "+udf.getClass().getSimpleName()+", is not supported");
   }
 
+  private VectorExpression getRandExpression(List<ExprNodeDesc> childExpr)
+    throws HiveException {
+
+    // prepare one output column
+    int outputCol = ocm.allocateOutputColumn("Double");
+    if (childExpr == null || childExpr.size() == 0) {
+
+      // make no-argument vectorized Rand expression
+      return new FuncRand(outputCol);
+    } else if (childExpr.size() == 1) {
+
+      // Make vectorized Rand expression with seed
+      long seed = getLongScalar(childExpr.get(0));
+      return new FuncRand(seed, outputCol);
+    }
+
+    throw new HiveException("Vectorization error. Rand has more than 1 argument.");
+  }
+
+  private VectorExpression getRoundExpression(List<ExprNodeDesc> childExpr)
+    throws HiveException {
+
+    // Handle one-argument case
+    if (childExpr.size() == 1) {
+      return getUnaryFunctionExpression("FuncRound", "Double", childExpr,
+          GENERATED_EXPR_PACKAGE);
+    }
+
+    // Handle two-argument case
+
+    // Try to get the second argument (the number of digits)
+    long numDigits = getLongScalar(childExpr.get(1));
+
+    // Use the standard logic for a unary function to handle the first argument.
+    VectorExpression e = getUnaryFunctionExpression("RoundWithNumDigits", "Double", childExpr,
+        CUSTOM_EXPR_PACKAGE);
+
+    // Set second argument for this special case
+    ((ISetLongArg) e).setArg(numDigits);
+    return e;
+  }
+
+  private VectorExpression getPowerExpression(List<ExprNodeDesc> childExpr)
+      throws HiveException {
+    String argType = childExpr.get(0).getTypeString();
+
+    // Try to get the second argument, typically a constant value (the power).
+    double power = getDoubleScalar(childExpr.get(1));
+
+    // Use the standard logic for a unary function to handle the first argument.
+    VectorExpression e = getUnaryFunctionExpression("FuncPower", "Double", childExpr,
+        CUSTOM_EXPR_PACKAGE);
+
+    // Set the second argument for this special case
+    ((ISetDoubleArg) e).setArg(power);
+    return e;
+  }
+
+  private VectorExpression getLogWithBaseExpression(List<ExprNodeDesc> childExpr)
+      throws HiveException {
+    if (childExpr.size() == 1) {
+
+      // No base provided, so this is equivalent to Ln
+      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));
+
+      // Use the standard logic for a unary function to handle the first argument.
+      VectorExpression e = getUnaryFunctionExpression("FuncLogWithBase", "Double", childExpr,
+          CUSTOM_EXPR_PACKAGE);
+
+      // set the second argument for this special case
+      ((ISetDoubleArg) e).setArg(base);
+      return e;
+    }
+
+    throw new HiveException("Udf: Log could not be vectorized");
+  }
+
+  private double getDoubleScalar(ExprNodeDesc expr) throws HiveException {
+    if (!(expr instanceof ExprNodeConstantDesc)) {
+      throw new HiveException("Constant value expected for UDF argument. " +
+          "Non-constant argument not supported for vectorization.");
+    }
+    ExprNodeConstantDesc constExpr = (ExprNodeConstantDesc) expr;
+    Object obj = getScalarValue(constExpr);
+    if (obj instanceof Double) {
+      return ((Double) obj).doubleValue();
+    } else if (obj instanceof DoubleWritable) {
+      return ((DoubleWritable) obj).get();
+    } else if (obj instanceof Integer) {
+      return (double) ((Integer) obj).longValue();
+    } else if (obj instanceof IntWritable) {
+      return (double) ((IntWritable) obj).get();
+    }
+
+    throw new HiveException("Udf: unhandled constant type for scalar argument."
+        + "Expecting double or integer");
+  }
+
+  private long getLongScalar(ExprNodeDesc expr) throws HiveException {
+    if (!(expr instanceof ExprNodeConstantDesc)) {
+      throw new HiveException("Constant value expected for UDF argument. " +
+          "Non-constant argument not supported for vectorization.");
+    }
+    ExprNodeConstantDesc constExpr = (ExprNodeConstantDesc) expr;
+    Object obj = getScalarValue(constExpr);
+    if (obj instanceof Integer) {
+      return (long) ((Integer) obj).longValue();
+    } else if (obj instanceof IntWritable) {
+      return (long) ((IntWritable) obj).get();
+    } else if (obj instanceof Long) {
+      return ((Long) obj).longValue();
+    } else if (obj instanceof LongWritable) {
+      return ((LongWritable) obj).get();
+    }
+
+    throw new HiveException("Udf: unhandled constant type for scalar argument."
+        + "Expecting integer or bigint");
+  }
+
   /* Return a vector expression for string concatenation, including the column-scalar,
    * scalar-column, and column-column cases.
    */
@@ -708,6 +1009,16 @@ public class VectorizationContext {
       String resultType, // result type name
       List<ExprNodeDesc> childExprList) throws HiveException {
 
+      return getUnaryExpression(vectorExprClassName, resultType, childExprList,
+          CUSTOM_EXPR_PACKAGE);
+  }
+
+  private VectorExpression getUnaryExpression(String vectorExprClassName,
+      String resultType,           // result type name
+      List<ExprNodeDesc> childExprList,
+      String packagePathPrefix     // prefix of package path name
+      ) throws HiveException {
+
     /* Create an instance of the class vectorExprClassName for the input column or expression result
      * and return it.
      */
@@ -722,13 +1033,12 @@ public class VectorizationContext {
       ExprNodeColumnDesc colDesc = (ExprNodeColumnDesc) childExpr;
       inputCol = getInputColumnIndex(colDesc.getColumn());
     } else {
-      // TODO? add code to handle constant argument case
+      // constant argument case not supported
       throw new HiveException("Expression not supported: "+childExpr);
     }
     String outputColumnType = getNormalizedTypeName(resultType);
     int outputCol = ocm.allocateOutputColumn(outputColumnType);
-    String className = "org.apache.hadoop.hive.ql.exec.vector.expressions."
-       + vectorExprClassName;
+    String className = packagePathPrefix + "." + vectorExprClassName;
     VectorExpression expr;
     try {
       expr = (VectorExpression) getConstructor(className).newInstance(inputCol, outputCol);
@@ -742,6 +1052,7 @@ public class VectorizationContext {
     return expr;
   }
 
+
   private VectorExpression getSubstrExpression(
       List<ExprNodeDesc> childExprList) throws HiveException {
 

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncBin.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncBin.java?rev=1530824&r1=1530823&r2=1530824&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncBin.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncBin.java Wed Oct  9 22:58:40 2013
@@ -26,11 +26,11 @@ import org.apache.hadoop.hive.ql.exec.ve
 public class FuncBin extends FuncLongToString {
   private static final long serialVersionUID = 1L;
 
-  FuncBin(int inputCol, int outputCol) {
+  public FuncBin(int inputCol, int outputCol) {
     super(inputCol, outputCol);
   }
 
-  FuncBin() {
+  public FuncBin() {
     super();
   }
 

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncHex.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncHex.java?rev=1530824&r1=1530823&r2=1530824&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncHex.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncHex.java Wed Oct  9 22:58:40 2013
@@ -24,11 +24,11 @@ import org.apache.hadoop.hive.ql.exec.ve
 public class FuncHex extends FuncLongToString {
   private static final long serialVersionUID = 1L;
 
-  FuncHex(int inputCol, int outputCol) {
+  public FuncHex(int inputCol, int outputCol) {
     super(inputCol, outputCol);
   }
 
-  FuncHex() {
+  public FuncHex() {
     super();
   }
 

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncLogWithBaseDoubleToDouble.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncLogWithBaseDoubleToDouble.java?rev=1530824&r1=1530823&r2=1530824&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncLogWithBaseDoubleToDouble.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncLogWithBaseDoubleToDouble.java Wed Oct  9 22:58:40 2013
@@ -18,22 +18,23 @@
 
 package org.apache.hadoop.hive.ql.exec.vector.expressions;
 
-public class FuncLogWithBaseDoubleToDouble extends MathFuncDoubleToDouble {
+
+public class FuncLogWithBaseDoubleToDouble extends MathFuncDoubleToDouble
+    implements ISetDoubleArg {
   private static final long serialVersionUID = 1L;
 
   private double base;
 
-  FuncLogWithBaseDoubleToDouble(int colNum, double base, int outputColumn) {
+  public FuncLogWithBaseDoubleToDouble(int colNum, int outputColumn) {
     super(colNum, outputColumn);
-    this.base = base;
   }
 
-  FuncLogWithBaseDoubleToDouble() {
+  public FuncLogWithBaseDoubleToDouble() {
     super();
   }
 
   @Override
-  double func(double d) {
+  protected double func(double d) {
     return Math.log(d) / Math.log(base);
   }
 
@@ -44,4 +45,10 @@ public class FuncLogWithBaseDoubleToDoub
   public void setBase(double base) {
     this.base = base;
   }
+
+  // used to set the second argument to function (a constant base)
+  @Override
+  public void setArg(double d) {
+    this.base = d;
+  }
 }

Added: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncLogWithBaseLongToDouble.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncLogWithBaseLongToDouble.java?rev=1530824&view=auto
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncLogWithBaseLongToDouble.java (added)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncLogWithBaseLongToDouble.java Wed Oct  9 22:58:40 2013
@@ -0,0 +1,54 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.ql.exec.vector.expressions;
+
+
+public class FuncLogWithBaseLongToDouble extends MathFuncLongToDouble
+    implements ISetDoubleArg {
+  private static final long serialVersionUID = 1L;
+
+  private double base;
+
+  public FuncLogWithBaseLongToDouble(int colNum, int outputColumn) {
+    super(colNum, outputColumn);
+  }
+
+  public FuncLogWithBaseLongToDouble() {
+    super();
+  }
+
+  @Override
+  protected double func(long l) {
+    return Math.log((double) l) / Math.log(base);
+  }
+
+  public double getBase() {
+    return base;
+  }
+
+  public void setBase(double base) {
+    this.base = base;
+  }
+
+  // used to set the second argument to function (a constant base)
+  @Override
+  public void setArg(double d) {
+    this.base = d;
+  }
+}

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncPowerDoubleToDouble.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncPowerDoubleToDouble.java?rev=1530824&r1=1530823&r2=1530824&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncPowerDoubleToDouble.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncPowerDoubleToDouble.java Wed Oct  9 22:58:40 2013
@@ -18,20 +18,22 @@
 
 package org.apache.hadoop.hive.ql.exec.vector.expressions;
 
+import org.apache.hadoop.hive.ql.exec.vector.DoubleColumnVector;
+
 /**
  * Vectorized implementation for Pow(a, power) and Power(a, power)
  */
-public class FuncPowerDoubleToDouble extends MathFuncDoubleToDouble {
+public class FuncPowerDoubleToDouble extends MathFuncDoubleToDouble
+    implements ISetDoubleArg {
   private static final long serialVersionUID = 1L;
 
   private double power;
 
-  FuncPowerDoubleToDouble(int colNum, double power, int outputColumn) {
+  public FuncPowerDoubleToDouble(int colNum, int outputColumn) {
     super(colNum, outputColumn);
-    this.power = power;
   }
 
-  FuncPowerDoubleToDouble() {
+  public FuncPowerDoubleToDouble() {
     super();
   }
 
@@ -47,4 +49,16 @@ public class FuncPowerDoubleToDouble ext
   public void setPower(double power) {
     this.power = power;
   }
+
+  // set the second argument (the power)
+  @Override
+  public void setArg(double d) {
+    this.power = d;
+  }
+
+  @Override
+  protected void cleanup(DoubleColumnVector outputColVector, int[] sel,
+      boolean selectedInUse, int n) {
+    // do nothing
+  }
 }

Added: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncPowerLongToDouble.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncPowerLongToDouble.java?rev=1530824&view=auto
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncPowerLongToDouble.java (added)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncPowerLongToDouble.java Wed Oct  9 22:58:40 2013
@@ -0,0 +1,64 @@
+/**
+ * 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;
+
+/**
+ * Vectorized implementation for Pow(a, power) and Power(a, power)
+ */
+public class FuncPowerLongToDouble extends MathFuncLongToDouble
+    implements ISetDoubleArg {
+  private static final long serialVersionUID = 1L;
+
+  private double power;
+
+  public FuncPowerLongToDouble(int colNum, int outputColumn) {
+    super(colNum, outputColumn);
+  }
+
+  public FuncPowerLongToDouble() {
+    super();
+  }
+
+  @Override
+  public double func(long l) {
+    return Math.pow((double) l, power);
+  }
+
+  public double getPower() {
+    return power;
+  }
+
+  public void setPower(double power) {
+    this.power = power;
+  }
+
+  // set the second argument (the power)
+  @Override
+  public void setArg(double d) {
+    this.power = d;
+  }
+
+  @Override
+  protected void cleanup(DoubleColumnVector outputColVector, int[] sel,
+      boolean selectedInUse, int n) {
+    // do nothing
+  }
+}

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncRand.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncRand.java?rev=1530824&r1=1530823&r2=1530824&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncRand.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncRand.java Wed Oct  9 22:58:40 2013
@@ -31,17 +31,17 @@ public class FuncRand extends VectorExpr
   private int outputCol;
   private Random random;
 
-  FuncRand(int outputCol) {
+  public FuncRand(int outputCol) {
     this.outputCol = outputCol;
     random = null;
   }
 
-  FuncRand(long seed, int outputCol) {
+  public FuncRand(long seed, int outputCol) {
     this.outputCol = outputCol;
-    random = new Random(seed);
+    this.random = new Random(seed);
   }
 
-  FuncRand() {
+  public FuncRand() {
   }
 
   @Override

Added: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/ISetDoubleArg.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/ISetDoubleArg.java?rev=1530824&view=auto
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/ISetDoubleArg.java (added)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/ISetDoubleArg.java Wed Oct  9 22:58:40 2013
@@ -0,0 +1,24 @@
+/**
+ * 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;
+
+// used to set the double precision constant argument to function (e.g. a constant base)
+public interface ISetDoubleArg {
+  void setArg(double d);
+}

Added: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/ISetLongArg.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/ISetLongArg.java?rev=1530824&view=auto
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/ISetLongArg.java (added)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/ISetLongArg.java Wed Oct  9 22:58:40 2013
@@ -0,0 +1,26 @@
+/**
+ * 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;
+
+/* Used to set the long constant argument to function
+ * (e.g. a constant number of digits to round to)
+ */
+public interface ISetLongArg {
+  void setArg(long l);
+}

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/MathExpr.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/MathExpr.java?rev=1530824&r1=1530823&r2=1530824&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/MathExpr.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/MathExpr.java Wed Oct  9 22:58:40 2013
@@ -18,12 +18,14 @@
 
 package org.apache.hadoop.hive.ql.exec.vector.expressions;
 
-/** 
+import org.apache.hadoop.hive.ql.exec.vector.DoubleColumnVector;
+
+/**
  * Math expression evaluation helper functions.
  * Some of these are referenced from ColumnUnaryFunc.txt.
  */
 public class MathExpr {
-  
+
   // Round using the "half-up" method used in Hive.
   public static double round(double d) {
     if (d > 0.0) {
@@ -32,15 +34,15 @@ public class MathExpr {
       return (double) ((long) (d - 0.5d));
     }
   }
-  
+
   public static double log2(double d) {
     return Math.log(d) / Math.log(2);
   }
-  
+
   public static long abs(long v) {
     return v >= 0 ? v : -v;
   }
-  
+
   public static double sign(double v) {
     return v >= 0 ? 1.0 : -1.0;
   }
@@ -48,4 +50,58 @@ public class MathExpr {
   public static double sign(long v) {
     return v >= 0 ? 1.0 : -1.0;
   }
+
+  // Convert all NaN values in vector v to NULL. Should only be used if n > 0.
+  public static void NaNToNull(DoubleColumnVector v, int[] sel, boolean selectedInUse, int n) {
+
+    // handle repeating case
+    if (v.isRepeating) {
+      if (Double.isNaN(v.vector[0])){
+        v.isNull[0] = true;
+        v.noNulls = false;
+      }
+      return;
+    }
+
+    if (v.noNulls) {
+      if (selectedInUse) {
+        for(int j = 0; j != n; j++) {
+          int i = sel[j];
+          if (Double.isNaN(v.vector[i])) {
+            v.isNull[i] = true;
+            v.noNulls = false;
+          } else {
+
+            // Must set isNull[i] to false to make sure
+            // it gets initialized, in case we set noNulls to true.
+            v.isNull[i] = false;
+          }
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          if (Double.isNaN(v.vector[i])) {
+            v.isNull[i] = true;
+            v.noNulls = false;
+          } else {
+            v.isNull[i] = false;
+          }
+        }
+      }
+    } else {  // there are nulls, so null array entries are already initialized
+      if (selectedInUse) {
+        for (int j = 0; j != n; j++) {
+          int i = sel[j];
+          if(Double.isNaN(v.vector[i])) {
+            v.isNull[i] = true;
+          }
+        }
+      } else {
+        for (int i = 0; i != n; i++) {
+          if(Double.isNaN(v.vector[i])) {
+            v.isNull[i] = true;
+          }
+        }
+      }
+    }
+  }
 }

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/MathFuncDoubleToDouble.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/MathFuncDoubleToDouble.java?rev=1530824&r1=1530823&r2=1530824&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/MathFuncDoubleToDouble.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/MathFuncDoubleToDouble.java Wed Oct  9 22:58:40 2013
@@ -27,7 +27,7 @@ import org.apache.hadoop.hive.ql.exec.ve
  * constant argument(s)) and returns long.
  * May be used for functions like ROUND(d, N), Pow(a, p) etc.
  *
- * Do NOT use this for simple math functions lone sin/cos/exp etc. that just take
+ * 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.
  */
@@ -38,14 +38,14 @@ public abstract class MathFuncDoubleToDo
   private int outputColumn;
 
   // Subclasses must override this with a function that implements the desired logic.
-  abstract double func(double d);
+  protected abstract double func(double d);
 
-  MathFuncDoubleToDouble(int colNum, int outputColumn) {
+  public MathFuncDoubleToDouble(int colNum, int outputColumn) {
     this.colNum = colNum;
     this.outputColumn = outputColumn;
   }
 
-  MathFuncDoubleToDouble() {
+  public MathFuncDoubleToDouble() {
   }
 
   @Override
@@ -103,6 +103,13 @@ public abstract class MathFuncDoubleToDo
       }
       outputColVector.isRepeating = false;
     }
+    cleanup(outputColVector, sel, batch.selectedInUse, n);
+  }
+
+  // override this with a no-op if subclass doesn't need to treat NaN as null
+  protected void cleanup(DoubleColumnVector outputColVector, int[] sel,
+      boolean selectedInUse, int n) {
+    MathExpr.NaNToNull(outputColVector, sel, selectedInUse, n);
   }
 
   @Override

Added: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/MathFuncLongToDouble.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/MathFuncLongToDouble.java?rev=1530824&view=auto
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/MathFuncLongToDouble.java (added)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/MathFuncLongToDouble.java Wed Oct  9 22:58:40 2013
@@ -0,0 +1,136 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.ql.exec.vector.expressions;
+
+import org.apache.hadoop.hive.ql.exec.vector.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 double (and optionally additional
+ * constant argument(s)) and returns long.
+ * May be used for functions like ROUND(d, N), Pow(a, p) 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 MathFuncLongToDouble 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 double func(long l);
+
+  public MathFuncLongToDouble(int colNum, int outputColumn) {
+    this.colNum = colNum;
+    this.outputColumn = outputColumn;
+  }
+
+  public MathFuncLongToDouble() {
+  }
+
+  @Override
+  public void evaluate(VectorizedRowBatch batch) {
+
+    if (childExpressions != null) {
+      this.evaluateChildren(batch);
+    }
+
+    LongColumnVector inputColVector = (LongColumnVector) batch.cols[colNum];
+    DoubleColumnVector outputColVector = (DoubleColumnVector) 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;
+    double[] 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;
+    }
+    cleanup(outputColVector, sel, batch.selectedInUse, n);
+  }
+
+  // override this with a no-op if subclass doesn't need to treat NaN as null
+  protected void cleanup(DoubleColumnVector outputColVector, int[] sel,
+      boolean selectedInUse, int n) {
+    MathExpr.NaNToNull(outputColVector, sel, selectedInUse, n);
+  }
+
+  @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 "double";
+  }
+}

Added: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/RoundWithNumDigitsDoubleToDouble.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/RoundWithNumDigitsDoubleToDouble.java?rev=1530824&view=auto
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/RoundWithNumDigitsDoubleToDouble.java (added)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/RoundWithNumDigitsDoubleToDouble.java Wed Oct  9 22:58:40 2013
@@ -0,0 +1,74 @@
+/**
+ * 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.udf.UDFRound;
+import org.apache.hadoop.hive.serde2.io.DoubleWritable;
+import org.apache.hadoop.io.IntWritable;
+
+// Vectorized implementation of ROUND(Col, N) function
+public class RoundWithNumDigitsDoubleToDouble extends MathFuncDoubleToDouble
+    implements ISetLongArg {
+  private static final long serialVersionUID = 1L;
+
+  private IntWritable decimalPlaces;
+  private transient UDFRound roundFunc;
+  private transient DoubleWritable dw;
+
+  public RoundWithNumDigitsDoubleToDouble(int colNum, int outputColumn) {
+    super(colNum, outputColumn);
+    this.decimalPlaces = new IntWritable();
+    roundFunc = new UDFRound();
+    dw = new DoubleWritable();
+  }
+
+  public RoundWithNumDigitsDoubleToDouble() {
+    super();
+    dw = new DoubleWritable();
+    roundFunc = new UDFRound();
+  }
+
+  // Round to the specified number of decimal places using the standard Hive round function.
+  @Override
+  public double func(double d) {
+    dw.set(d);
+    return roundFunc.evaluate(dw, decimalPlaces).get();
+  }
+
+  void setDecimalPlaces(IntWritable decimalPlaces) {
+    this.decimalPlaces = decimalPlaces;
+  }
+
+  IntWritable getDecimalPlaces() {
+    return this.decimalPlaces;
+  }
+
+  void setRoundFunc(UDFRound roundFunc) {
+    this.roundFunc = roundFunc;
+  }
+
+  UDFRound getRoundFunc() {
+    return this.roundFunc;
+  }
+
+  @Override
+  public void setArg(long l) {
+    this.decimalPlaces.set((int) l);
+  }
+}

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=1530824&r1=1530823&r2=1530824&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 Wed Oct  9 22:58:40 2013
@@ -68,11 +68,26 @@ import org.apache.hadoop.hive.ql.plan.Ma
 import org.apache.hadoop.hive.ql.plan.OperatorDesc;
 import org.apache.hadoop.hive.ql.plan.PartitionDesc;
 import org.apache.hadoop.hive.ql.plan.api.OperatorType;
+import org.apache.hadoop.hive.ql.udf.UDFAcos;
+import org.apache.hadoop.hive.ql.udf.UDFAsin;
+import org.apache.hadoop.hive.ql.udf.UDFAtan;
+import org.apache.hadoop.hive.ql.udf.UDFBin;
+import org.apache.hadoop.hive.ql.udf.UDFCeil;
+import org.apache.hadoop.hive.ql.udf.UDFConv;
+import org.apache.hadoop.hive.ql.udf.UDFCos;
 import org.apache.hadoop.hive.ql.udf.UDFDayOfMonth;
+import org.apache.hadoop.hive.ql.udf.UDFDegrees;
+import org.apache.hadoop.hive.ql.udf.UDFExp;
+import org.apache.hadoop.hive.ql.udf.UDFFloor;
+import org.apache.hadoop.hive.ql.udf.UDFHex;
 import org.apache.hadoop.hive.ql.udf.UDFHour;
 import org.apache.hadoop.hive.ql.udf.UDFLTrim;
 import org.apache.hadoop.hive.ql.udf.UDFLength;
 import org.apache.hadoop.hive.ql.udf.UDFLike;
+import org.apache.hadoop.hive.ql.udf.UDFLn;
+import org.apache.hadoop.hive.ql.udf.UDFLog;
+import org.apache.hadoop.hive.ql.udf.UDFLog10;
+import org.apache.hadoop.hive.ql.udf.UDFLog2;
 import org.apache.hadoop.hive.ql.udf.UDFMinute;
 import org.apache.hadoop.hive.ql.udf.UDFOPDivide;
 import org.apache.hadoop.hive.ql.udf.UDFOPMinus;
@@ -81,13 +96,23 @@ 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.UDFPower;
 import org.apache.hadoop.hive.ql.udf.UDFRTrim;
+import org.apache.hadoop.hive.ql.udf.UDFRadians;
+import org.apache.hadoop.hive.ql.udf.UDFRand;
+import org.apache.hadoop.hive.ql.udf.UDFRound;
 import org.apache.hadoop.hive.ql.udf.UDFSecond;
+import org.apache.hadoop.hive.ql.udf.UDFSign;
+import org.apache.hadoop.hive.ql.udf.UDFSin;
+import org.apache.hadoop.hive.ql.udf.UDFSqrt;
 import org.apache.hadoop.hive.ql.udf.UDFSubstr;
+import org.apache.hadoop.hive.ql.udf.UDFTan;
 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;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDFAbs;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDFBridge;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDFConcat;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDFLower;
@@ -168,9 +193,34 @@ public class Vectorizer implements Physi
     supportedGenericUDFs.add(UDFRTrim.class);
     supportedGenericUDFs.add(UDFTrim.class);
 
+    supportedGenericUDFs.add(UDFSin.class);
+    supportedGenericUDFs.add(UDFCos.class);
+    supportedGenericUDFs.add(UDFTan.class);
+    supportedGenericUDFs.add(UDFAsin.class);
+    supportedGenericUDFs.add(UDFAcos.class);
+    supportedGenericUDFs.add(UDFAtan.class);
+    supportedGenericUDFs.add(UDFDegrees.class);
+    supportedGenericUDFs.add(UDFRadians.class);
+    supportedGenericUDFs.add(UDFFloor.class);
+    supportedGenericUDFs.add(UDFCeil.class);
+    supportedGenericUDFs.add(UDFExp.class);
+    supportedGenericUDFs.add(UDFLn.class);
+    supportedGenericUDFs.add(UDFLog2.class);
+    supportedGenericUDFs.add(UDFLog10.class);
+    supportedGenericUDFs.add(UDFLog.class);
+    supportedGenericUDFs.add(UDFPower.class);
+    supportedGenericUDFs.add(UDFRound.class);
+    supportedGenericUDFs.add(UDFSqrt.class);
+    supportedGenericUDFs.add(UDFSign.class);
+    supportedGenericUDFs.add(UDFRand.class);
+    supportedGenericUDFs.add(UDFBin.class);
+    supportedGenericUDFs.add(UDFHex.class);
+    supportedGenericUDFs.add(UDFConv.class);
+
     supportedGenericUDFs.add(GenericUDFLower.class);
     supportedGenericUDFs.add(GenericUDFUpper.class);
     supportedGenericUDFs.add(GenericUDFConcat.class);
+    supportedGenericUDFs.add(GenericUDFAbs.class);
 
     supportedAggregationUdfs.add("min");
     supportedAggregationUdfs.add("max");

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=1530824&r1=1530823&r2=1530824&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 Wed Oct  9 22:58:40 2013
@@ -38,7 +38,7 @@ import org.junit.Test;
 
 
 public class TestVectorMathFunctions {
-  
+
   private static final double eps = 1.0e-7;
   private static boolean equalsWithinTolerance(double a, double b) {
     return Math.abs(a - b) < eps;
@@ -58,7 +58,7 @@ public class TestVectorMathFunctions {
     Assert.assertEquals(1d, resultV.vector[4]);
     Assert.assertEquals(1d, resultV.vector[5]);
     Assert.assertEquals(2d, resultV.vector[6]);
-    
+
     // spot check null propagation
     b.cols[0].noNulls = false;
     b.cols[0].isNull[3] = true;
@@ -66,28 +66,29 @@ public class TestVectorMathFunctions {
     expr.evaluate(b);
     Assert.assertEquals(true, resultV.isNull[3]);
     Assert.assertEquals(false, resultV.noNulls);
-    
+
     // check isRepeating propagation
     b.cols[0].isRepeating = true;
     resultV.isRepeating = false;
     expr.evaluate(b);
     Assert.assertEquals(-2d, resultV.vector[0]);
-    Assert.assertEquals(true, resultV.isRepeating);   
-    
+    Assert.assertEquals(true, resultV.isRepeating);
+
     resultV.isRepeating = false;
     b.cols[0].noNulls = true;
     expr.evaluate(b);
     Assert.assertEquals(-2d, resultV.vector[0]);
-    Assert.assertEquals(true, resultV.isRepeating);   
+    Assert.assertEquals(true, resultV.isRepeating);
   }
-  
+
   @Test
   public void testRoundToDecimalPlaces() {
     VectorizedRowBatch b = getVectorizedRowBatchDoubleInDoubleOut();
-    VectorExpression expr = new RoundDoubleToDoubleWithNumDigits(0, 4, 1);
+    VectorExpression expr = new RoundWithNumDigitsDoubleToDouble(0, 1);
+    ((ISetLongArg) expr).setArg(4);  // set number of digits
     expr.evaluate(b);
     DoubleColumnVector resultV = (DoubleColumnVector) b.cols[1];
-    
+
     // Verify result is rounded to 4 digits
     Assert.assertEquals(1.2346d, resultV.vector[7]);
   }
@@ -105,14 +106,14 @@ public class TestVectorMathFunctions {
     dcv.vector[4] = 0.5d;
     dcv.vector[5] = 0.7d;
     dcv.vector[6] = 1.5d;
-    
+
     batch.cols[0] = dcv;
     batch.cols[1] = lcv;
-      
+
     batch.size = 7;
     return batch;
   }
-  
+
   public static VectorizedRowBatch getVectorizedRowBatchDoubleInDoubleOut() {
     VectorizedRowBatch batch = new VectorizedRowBatch(2);
     DoubleColumnVector inV;
@@ -127,14 +128,14 @@ public class TestVectorMathFunctions {
     inV.vector[5] = 0.7d;
     inV.vector[6] = 1.5d;
     inV.vector[7] = 1.2345678d;
-    
+
     batch.cols[0] = inV;
     batch.cols[1] = outV;
-      
+
     batch.size = 8;
     return batch;
   }
-  
+
   public static VectorizedRowBatch getVectorizedRowBatchLongInDoubleOut() {
     VectorizedRowBatch batch = new VectorizedRowBatch(2);
     LongColumnVector lcv;
@@ -146,14 +147,14 @@ public class TestVectorMathFunctions {
     lcv.vector[2] = 0;
     lcv.vector[3] = 1;
     lcv.vector[4] = 2;
-    
+
     batch.cols[0] = lcv;
     batch.cols[1] = dcv;
-      
+
     batch.size = 5;
     return batch;
   }
-  
+
   public static VectorizedRowBatch getVectorizedRowBatchLongInLongOut() {
     VectorizedRowBatch batch = new VectorizedRowBatch(2);
     LongColumnVector inV, outV;
@@ -161,14 +162,14 @@ public class TestVectorMathFunctions {
     outV = new LongColumnVector();
     inV.vector[0] = -2;
     inV.vector[1] = 2;
- 
+
     batch.cols[0] = inV;
     batch.cols[1] = outV;
-      
+
     batch.size = 2;
     return batch;
   }
-  
+
   public static VectorizedRowBatch getBatchForStringMath() {
     VectorizedRowBatch batch = new VectorizedRowBatch(3);
     LongColumnVector inL;
@@ -184,7 +185,7 @@ public class TestVectorMathFunctions {
       inS.setVal(0, "00".getBytes("UTF-8"), 0, 2);
       inS.setVal(1, "3232".getBytes("UTF-8"), 0, 4);
       byte[] bad = "bad data".getBytes("UTF-8");
-      inS.setVal(2, bad, 0, bad.length);  
+      inS.setVal(2, bad, 0, bad.length);
     } catch (UnsupportedEncodingException e) {
       e.printStackTrace();
       Assert.assertTrue(false);
@@ -193,12 +194,12 @@ public class TestVectorMathFunctions {
     batch.cols[0] = inS;
     batch.cols[1] = inL;
     batch.cols[2] = outS;
-      
+
     batch.size = 3;
     return batch;
   }
-  
-  /* 
+
+  /*
    * The following tests spot-check that vectorized functions with signature
    * DOUBLE func(DOUBLE) that came from template ColumnUnaryFunc.txt
    * get the right result. Null propagation, isRepeating
@@ -214,7 +215,7 @@ public class TestVectorMathFunctions {
     expr.evaluate(b);
     Assert.assertEquals(Math.sin(0.5d), resultV.vector[4]);
   }
-  
+
   @Test
   public void testVectorCos() {
     VectorizedRowBatch b = getVectorizedRowBatchDoubleInDoubleOut();
@@ -224,7 +225,7 @@ public class TestVectorMathFunctions {
     expr.evaluate(b);
     Assert.assertEquals(Math.cos(0.5d), resultV.vector[4]);
   }
-  
+
   @Test
   public void testVectorTan() {
     VectorizedRowBatch b = getVectorizedRowBatchDoubleInDoubleOut();
@@ -234,7 +235,7 @@ public class TestVectorMathFunctions {
     expr.evaluate(b);
     Assert.assertEquals(Math.tan(0.5d), resultV.vector[4]);
   }
-  
+
   @Test
   public void testVectorASin() {
     VectorizedRowBatch b = getVectorizedRowBatchDoubleInDoubleOut();
@@ -244,7 +245,7 @@ public class TestVectorMathFunctions {
     expr.evaluate(b);
     Assert.assertEquals(Math.asin(0.5d), resultV.vector[4]);
   }
-  
+
   @Test
   public void testVectorACos() {
     VectorizedRowBatch b = getVectorizedRowBatchDoubleInDoubleOut();
@@ -254,7 +255,7 @@ public class TestVectorMathFunctions {
     expr.evaluate(b);
     Assert.assertEquals(Math.acos(0.5d), resultV.vector[4]);
   }
-  
+
   @Test
   public void testVectorATan() {
     VectorizedRowBatch b = getVectorizedRowBatchDoubleInDoubleOut();
@@ -264,7 +265,7 @@ public class TestVectorMathFunctions {
     expr.evaluate(b);
     Assert.assertEquals(Math.atan(0.5d), resultV.vector[4]);
   }
-  
+
   @Test
   public void testVectorDegrees() {
     VectorizedRowBatch b = getVectorizedRowBatchDoubleInDoubleOut();
@@ -274,7 +275,7 @@ public class TestVectorMathFunctions {
     expr.evaluate(b);
     Assert.assertEquals(Math.toDegrees(0.5d), resultV.vector[4]);
   }
-  
+
   @Test
   public void testVectorRadians() {
     VectorizedRowBatch b = getVectorizedRowBatchDoubleInDoubleOut();
@@ -284,7 +285,7 @@ public class TestVectorMathFunctions {
     expr.evaluate(b);
     Assert.assertEquals(Math.toRadians(0.5d), resultV.vector[4]);
   }
-  
+
   @Test
   public void testVectorFloor() {
     VectorizedRowBatch b = getVectorizedRowBatchDoubleInLongOut();
@@ -295,7 +296,7 @@ public class TestVectorMathFunctions {
     Assert.assertEquals(-2, resultV.vector[0]);
     Assert.assertEquals(1, resultV.vector[6]);
   }
-  
+
   @Test
   public void testVectorCeil() {
     VectorizedRowBatch b = getVectorizedRowBatchDoubleInLongOut();
@@ -306,7 +307,7 @@ public class TestVectorMathFunctions {
     Assert.assertEquals(-1, resultV.vector[0]);
     Assert.assertEquals(2, resultV.vector[6]);
   }
-  
+
   @Test
   public void testVectorExp() {
     VectorizedRowBatch b = getVectorizedRowBatchDoubleInDoubleOut();
@@ -316,10 +317,10 @@ public class TestVectorMathFunctions {
     expr.evaluate(b);
     Assert.assertEquals(Math.exp(0.5d), resultV.vector[4]);
   }
-  
+
   @Test
   public void testVectorLn() {
-    
+
     // test double->double version
     VectorizedRowBatch b = getVectorizedRowBatchDoubleInDoubleOut();
     DoubleColumnVector resultV = (DoubleColumnVector) b.cols[1];
@@ -327,7 +328,7 @@ public class TestVectorMathFunctions {
     VectorExpression expr = new FuncLnDoubleToDouble(0, 1);
     expr.evaluate(b);
     Assert.assertEquals(Math.log(0.5), resultV.vector[4]);
-    
+
     // test long->double version
     b = getVectorizedRowBatchLongInDoubleOut();
     resultV = (DoubleColumnVector) b.cols[1];
@@ -336,10 +337,10 @@ public class TestVectorMathFunctions {
     expr.evaluate(b);
     Assert.assertEquals(Math.log(2), resultV.vector[4]);
   }
-  
+
   @Test
   public void testVectorLog2() {
-    
+
     // test double->double version
     VectorizedRowBatch b = getVectorizedRowBatchDoubleInDoubleOut();
     DoubleColumnVector resultV = (DoubleColumnVector) b.cols[1];
@@ -347,7 +348,7 @@ public class TestVectorMathFunctions {
     VectorExpression expr = new FuncLog2DoubleToDouble(0, 1);
     expr.evaluate(b);
     Assert.assertEquals(Math.log(0.5d) / Math.log(2), resultV.vector[4]);
-    
+
     // test long->double version
     b = getVectorizedRowBatchLongInDoubleOut();
     resultV = (DoubleColumnVector) b.cols[1];
@@ -356,10 +357,10 @@ public class TestVectorMathFunctions {
     expr.evaluate(b);
     Assert.assertEquals(Math.log(1) / Math.log(2), resultV.vector[3]);
   }
-  
+
   @Test
   public void testVectorLog10() {
-    
+
     // test double->double version
     VectorizedRowBatch b = getVectorizedRowBatchDoubleInDoubleOut();
     DoubleColumnVector resultV = (DoubleColumnVector) b.cols[1];
@@ -367,7 +368,7 @@ public class TestVectorMathFunctions {
     VectorExpression expr = new FuncLog10DoubleToDouble(0, 1);
     expr.evaluate(b);
     Assert.assertTrue(equalsWithinTolerance(Math.log(0.5d) / Math.log(10), resultV.vector[4]));
-    
+
     // test long->double version
     b = getVectorizedRowBatchLongInDoubleOut();
     resultV = (DoubleColumnVector) b.cols[1];
@@ -376,7 +377,7 @@ public class TestVectorMathFunctions {
     expr.evaluate(b);
     Assert.assertEquals(Math.log(1) / Math.log(10), resultV.vector[3]);
   }
-  
+
   @Test
   public void testVectorRand() {
     VectorizedRowBatch b = new VectorizedRowBatch(1);
@@ -393,13 +394,13 @@ public class TestVectorMathFunctions {
       Assert.assertTrue(v.vector[i] >= 0.0 && v.vector[i] <= 1.0);
     }
     double avg = sum / n;
-    
+
     /* The random values must be between 0 and 1, distributed uniformly.
-     * So the average value of a large set should be about 0.5. Verify it is 
+     * So the average value of a large set should be about 0.5. Verify it is
      * close to this value.
      */
     Assert.assertTrue(avg > 0.3 && avg < 0.7);
-    
+
     // Now, test again with a seed.
     Arrays.fill(v.vector, 0);
     expr = new FuncRand(99999, 0);
@@ -412,29 +413,31 @@ public class TestVectorMathFunctions {
     avg = sum / n;
     Assert.assertTrue(avg > 0.3 && avg < 0.7);
   }
-  
+
   @Test
   public void testVectorLogBase() {
-    
+
     // test double->double version
     VectorizedRowBatch b = getVectorizedRowBatchDoubleInDoubleOut();
     DoubleColumnVector resultV = (DoubleColumnVector) b.cols[1];
     b.cols[0].noNulls = true;
-    VectorExpression expr = new FuncLogWithBaseDoubleToDouble(0, 10.0d, 1);
+    VectorExpression expr = new FuncLogWithBaseDoubleToDouble(0, 1);
+    ((ISetDoubleArg) expr).setArg(10.0d);  // set base
     expr.evaluate(b);
     Assert.assertTrue(equalsWithinTolerance(Math.log(0.5d) / Math.log(10), resultV.vector[4]));
   }
-  
+
   @Test
   public void testVectorPower() {
     VectorizedRowBatch b = getVectorizedRowBatchDoubleInDoubleOut();
     DoubleColumnVector resultV = (DoubleColumnVector) b.cols[1];
     b.cols[0].noNulls = true;
-    VectorExpression expr = new FuncPowerDoubleToDouble(0, 2.0d, 1);
+    VectorExpression expr = new FuncPowerDoubleToDouble(0, 1);
+    ((ISetDoubleArg) expr).setArg(2.0d);  // set power
     expr.evaluate(b);
     Assert.assertTrue(equalsWithinTolerance(0.5d * 0.5d, resultV.vector[4]));
   }
-  
+
   @Test
   public void testVectorSqrt() {
     VectorizedRowBatch b = getVectorizedRowBatchDoubleInDoubleOut();
@@ -444,10 +447,10 @@ public class TestVectorMathFunctions {
     expr.evaluate(b);
     Assert.assertEquals(Math.sqrt(0.5d), resultV.vector[4]);
   }
-  
+
   @Test
   public void testVectorAbs() {
-    
+
     // test double->double version
     VectorizedRowBatch b = getVectorizedRowBatchDoubleInDoubleOut();
     DoubleColumnVector resultV = (DoubleColumnVector) b.cols[1];
@@ -456,7 +459,7 @@ public class TestVectorMathFunctions {
     expr.evaluate(b);
     Assert.assertEquals(1.5, resultV.vector[0]);
     Assert.assertEquals(0.5, resultV.vector[4]);
-    
+
     // test long->long version
     b = getVectorizedRowBatchLongInLongOut();
     LongColumnVector resultVLong = (LongColumnVector) b.cols[1];
@@ -466,10 +469,10 @@ public class TestVectorMathFunctions {
     Assert.assertEquals(2, resultVLong.vector[0]);
     Assert.assertEquals(2, resultVLong.vector[1]);
   }
-  
+
   @Test
   public void testVectorSign() {
-    
+
     // test double->double version
     VectorizedRowBatch b = getVectorizedRowBatchDoubleInDoubleOut();
     DoubleColumnVector resultV = (DoubleColumnVector) b.cols[1];
@@ -478,7 +481,7 @@ public class TestVectorMathFunctions {
     expr.evaluate(b);
     Assert.assertEquals(-1.0d, resultV.vector[0]);
     Assert.assertEquals(1.0d, resultV.vector[4]);
-    
+
     // test long->double version
     b = getVectorizedRowBatchLongInDoubleOut();
     resultV = (DoubleColumnVector) b.cols[1];
@@ -486,84 +489,41 @@ public class TestVectorMathFunctions {
     expr = new FuncSignLongToDouble(0, 1);
     expr.evaluate(b);
     Assert.assertEquals(-1.0d, resultV.vector[0]);
-    Assert.assertEquals(1.0d, resultV.vector[4]);   
+    Assert.assertEquals(1.0d, resultV.vector[4]);
   }
-  
-  @Test 
+
+  @Test
   public void testVectorBin() {
-    
+
     // test conversion of long->string
     VectorizedRowBatch b = getBatchForStringMath();
     BytesColumnVector resultV = (BytesColumnVector) b.cols[2];
     b.cols[0].noNulls = true;
-    VectorExpression expr = new FuncBin(1, 2); 
+    VectorExpression expr = new FuncBin(1, 2);
     expr.evaluate(b);
     String s = new String(resultV.vector[1], resultV.start[1], resultV.length[1]);
     Assert.assertEquals("11111111", s);
   }
-  
-  @Test 
+
+  @Test
   public void testVectorHex() {
-    
+
     // test long->string version
     VectorizedRowBatch b = getBatchForStringMath();
     BytesColumnVector resultV = (BytesColumnVector) b.cols[2];
     b.cols[1].noNulls = true;
-    VectorExpression expr = new FuncHex(1, 2); 
+    VectorExpression expr = new FuncHex(1, 2);
     expr.evaluate(b);
     String s = new String(resultV.vector[1], resultV.start[1], resultV.length[1]);
     Assert.assertEquals("FF", s);
-    
+
     // test string->string version
     b = getBatchForStringMath();
     resultV = (BytesColumnVector) b.cols[2];
     b.cols[0].noNulls = true;
-    expr = new StringHex(0, 2); 
+    expr = new StringHex(0, 2);
     expr.evaluate(b);
     s = new String(resultV.vector[1], resultV.start[1], resultV.length[1]);
     Assert.assertEquals("33323332", s);
   }
-  
-  @Test
-  public void testVectorUnhex() {
-    
-    // test string->string version
-    VectorizedRowBatch b = getBatchForStringMath();
-    BytesColumnVector resultV = (BytesColumnVector) b.cols[2];
-    b.cols[0].noNulls = true;
-    resultV.noNulls = true;
-    VectorExpression expr = new StringUnhex(0, 2); 
-    expr.evaluate(b);
-    String s = new String(resultV.vector[1], resultV.start[1], resultV.length[1]);
-    Assert.assertEquals("22", s);
-    
-    // check for NULL output for entry 2 ("bad data")
-    Assert.assertEquals(false, resultV.noNulls);
-    Assert.assertEquals(true, resultV.isNull[2]);
-  }
-  
-  @Test
-  public void testVectorConv() {
-    
-    // test string->string version
-    VectorizedRowBatch b = getBatchForStringMath();
-    BytesColumnVector resultV = (BytesColumnVector) b.cols[2];
-    b.cols[0].noNulls = true;
-    resultV.noNulls = true;
-    VectorExpression expr = new StringConv(0, 2, 10, 16); 
-    expr.evaluate(b);
-    String s[] = new String[3];
-    try {
-      s[0] = new String(resultV.vector[0], resultV.start[0], resultV.length[0], "UTF-8");
-      s[1] = new String(resultV.vector[1], resultV.start[1], resultV.length[1], "UTF-8");
-    } catch (UnsupportedEncodingException e) {
-      e.printStackTrace();
-    }
-    Assert.assertEquals("0", s[0]);
-    Assert.assertEquals("CA0", s[1]);
-    
-    // Note: the documentation says that "bad data" should translate to NULL output,
-    // but that is not the case in the existing code, which returns "0". So, this test skips 
-    // checking output for element 2.
-  }
 }