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/09/25 16:13:37 UTC

svn commit: r1526183 - in /hive/branches/vectorization: ant/src/org/apache/hadoop/hive/ant/ ql/src/gen/vectorization/ExpressionTemplates/ ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/ ql/src/java/org/apache/hadoop/hive/ql/udf/ ql/src/t...

Author: hashutosh
Date: Wed Sep 25 14:13:37 2013
New Revision: 1526183

URL: http://svn.apache.org/r1526183
Log:
HIVE-4822 : implement vectorized math functions (Eric Hanson via Ashutosh Chauhan)

Added:
    hive/branches/vectorization/ql/src/gen/vectorization/ExpressionTemplates/ColumnUnaryFunc.txt
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncBin.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncHex.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncLogWithBaseDoubleToDouble.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncLongToString.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncPowerDoubleToDouble.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncRand.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/MathExpr.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/MathFuncDoubleToDouble.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/RoundDoubleToDoubleWithNumDigits.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/StringConv.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/StringHex.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/StringUnhex.java
    hive/branches/vectorization/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorMathFunctions.java
Modified:
    hive/branches/vectorization/ant/src/org/apache/hadoop/hive/ant/GenVectorCode.java
    hive/branches/vectorization/ql/src/gen/vectorization/ExpressionTemplates/ColumnUnaryMinus.txt
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/StringUnaryUDF.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFHex.java
    hive/branches/vectorization/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorStringExpressions.java

Modified: hive/branches/vectorization/ant/src/org/apache/hadoop/hive/ant/GenVectorCode.java
URL: http://svn.apache.org/viewvc/hive/branches/vectorization/ant/src/org/apache/hadoop/hive/ant/GenVectorCode.java?rev=1526183&r1=1526182&r2=1526183&view=diff
==============================================================================
--- hive/branches/vectorization/ant/src/org/apache/hadoop/hive/ant/GenVectorCode.java (original)
+++ hive/branches/vectorization/ant/src/org/apache/hadoop/hive/ant/GenVectorCode.java Wed Sep 25 14:13:37 2013
@@ -216,6 +216,43 @@ public class GenVectorCode extends Task 
         {"FilterColumnCompareColumn", "GreaterEqual", "long", "long", ">="},
         {"FilterColumnCompareColumn", "GreaterEqual", "double", "long", ">="},
 
+      // template, <ClassNamePrefix>, <ReturnType>, <OperandType>, <FuncName>, <OperandCast>,
+      //   <ResultCast>
+      {"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)", ""},
+      // 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)", ""},
+      // 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", "", ""},
+
+
         {"ColumnUnaryMinus", "long"},
         {"ColumnUnaryMinus", "double"},
 
@@ -355,6 +392,8 @@ public class GenVectorCode extends Task 
         generateColumnArithmeticColumn(tdesc);
       } else if (tdesc[0].equals("ColumnUnaryMinus")) {
         generateColumnUnaryMinus(tdesc);
+      } else if (tdesc[0].equals("ColumnUnaryFunc")) {
+        generateColumnUnaryFunc(tdesc);
       } else if (tdesc[0].equals("VectorUDAFMinMax")) {
         generateVectorUDAFMinMax(tdesc);
       } else if (tdesc[0].equals("VectorUDAFMinMaxString")) {
@@ -541,6 +580,33 @@ public class GenVectorCode extends Task 
     writeFile(outputFile, templateString);
   }
 
+  // template, <ClassNamePrefix>, <ReturnType>, <OperandType>, <FuncName>, <OperandCast>, <ResultCast>
+  private void generateColumnUnaryFunc(String[] tdesc) throws IOException {
+    String classNamePrefix = tdesc[1];
+    String operandType = tdesc[3];
+    String inputColumnVectorType = this.getColumnVectorType(operandType);
+    String returnType = tdesc[2];
+    String outputColumnVectorType = this.getColumnVectorType(returnType);
+    String className = classNamePrefix + getCamelCaseType(operandType) + "To"
+      + getCamelCaseType(returnType);
+    String outputFile = joinPath(this.expressionOutputDirectory, className + ".java");
+    String templateFile = joinPath(this.expressionTemplateDirectory, tdesc[0] + ".txt");
+    String templateString = readFile(templateFile);
+    String funcName = tdesc[4];
+    String operandCast = tdesc[5];
+    String resultCast = tdesc[6];
+    // Expand, and write result
+    templateString = templateString.replaceAll("<ClassName>", className);
+    templateString = templateString.replaceAll("<InputColumnVectorType>", inputColumnVectorType);
+    templateString = templateString.replaceAll("<OutputColumnVectorType>", outputColumnVectorType);
+    templateString = templateString.replaceAll("<OperandType>", operandType);
+    templateString = templateString.replaceAll("<ReturnType>", returnType);
+    templateString = templateString.replaceAll("<FuncName>", funcName);
+    templateString = templateString.replaceAll("<OperandCast>", operandCast);
+    templateString = templateString.replaceAll("<ResultCast>", resultCast);
+    writeFile(outputFile, templateString);
+  }
+
   private void generateColumnArithmeticColumn(String [] tdesc) throws IOException {
     String operatorName = tdesc[1];
     String operandType1 = tdesc[2];

Added: hive/branches/vectorization/ql/src/gen/vectorization/ExpressionTemplates/ColumnUnaryFunc.txt
URL: http://svn.apache.org/viewvc/hive/branches/vectorization/ql/src/gen/vectorization/ExpressionTemplates/ColumnUnaryFunc.txt?rev=1526183&view=auto
==============================================================================
--- hive/branches/vectorization/ql/src/gen/vectorization/ExpressionTemplates/ColumnUnaryFunc.txt (added)
+++ hive/branches/vectorization/ql/src/gen/vectorization/ExpressionTemplates/ColumnUnaryFunc.txt Wed Sep 25 14:13:37 2013
@@ -0,0 +1,121 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+ 
+package org.apache.hadoop.hive.ql.exec.vector.expressions.gen;
+
+import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.MathExpr;
+import org.apache.hadoop.hive.ql.exec.vector.*;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+
+public class <ClassName> extends VectorExpression {
+  private static final long serialVersionUID = 1L;
+
+  private int colNum;
+  private int outputColumn;
+
+  public <ClassName>(int colNum, int outputColumn) {
+    this();
+    this.colNum = colNum;
+    this.outputColumn = outputColumn;
+  }
+  
+  public <ClassName>() {
+    super();
+  }
+
+  @Override
+  public void evaluate(VectorizedRowBatch batch) {
+
+    if (childExpressions != null) {
+      this.evaluateChildren(batch);
+    }
+
+    <InputColumnVectorType> inputColVector = (<InputColumnVectorType>) batch.cols[colNum];
+    <OutputColumnVectorType> outputColVector = (<OutputColumnVectorType>) batch.cols[outputColumn];
+    int[] sel = batch.selected;
+    boolean[] inputIsNull = inputColVector.isNull;
+    boolean[] outputIsNull = outputColVector.isNull;
+    outputColVector.noNulls = inputColVector.noNulls;
+    int n = batch.size;
+    <OperandType>[] vector = inputColVector.vector;
+    <ReturnType>[] outputVector = outputColVector.vector;
+    
+    // return immediately if batch is empty
+    if (n == 0) {
+      return;
+    }
+
+    if (inputColVector.isRepeating) {
+      //All must be selected otherwise size would be zero
+      //Repeating property will not change.
+      outputVector[0] = <ResultCast> <FuncName>(<OperandCast> 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] = <ResultCast> <FuncName>(<OperandCast> vector[i]);
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = <ResultCast> <FuncName>(<OperandCast> 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] = <ResultCast> <FuncName>(<OperandCast> vector[i]);
+          outputIsNull[i] = inputIsNull[i];
+	    }
+      } else {
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = <ResultCast> <FuncName>(<OperandCast> vector[i]);
+        }
+        System.arraycopy(inputIsNull, 0, outputIsNull, 0, n);
+      }
+      outputColVector.isRepeating = false;
+    }
+  }
+
+  @Override
+  public int getOutputColumn() {
+    return outputColumn;
+  }
+  
+  @Override
+  public String getOutputType() {
+    return "<ReturnType>";
+  }
+  
+  public int getColNum() {
+    return colNum;
+  }
+
+  public void setColNum(int colNum) {
+    this.colNum = colNum;
+  }
+
+  public void setOutputColumn(int outputColumn) {
+    this.outputColumn = outputColumn;
+  }
+}

Modified: hive/branches/vectorization/ql/src/gen/vectorization/ExpressionTemplates/ColumnUnaryMinus.txt
URL: http://svn.apache.org/viewvc/hive/branches/vectorization/ql/src/gen/vectorization/ExpressionTemplates/ColumnUnaryMinus.txt?rev=1526183&r1=1526182&r2=1526183&view=diff
==============================================================================
--- hive/branches/vectorization/ql/src/gen/vectorization/ExpressionTemplates/ColumnUnaryMinus.txt (original)
+++ hive/branches/vectorization/ql/src/gen/vectorization/ExpressionTemplates/ColumnUnaryMinus.txt Wed Sep 25 14:13:37 2013
@@ -33,11 +33,13 @@ public class <ClassName> extends VectorE
   private int outputColumn;
 
   public <ClassName>(int colNum, int outputColumn) {
+    this();
     this.colNum = colNum;
     this.outputColumn = outputColumn;
   }
 
   public <ClassName>() {
+    super();
   }
 
   @Override

Added: hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncBin.java
URL: http://svn.apache.org/viewvc/hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncBin.java?rev=1526183&view=auto
==============================================================================
--- hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncBin.java (added)
+++ hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncBin.java Wed Sep 25 14:13:37 2013
@@ -0,0 +1,49 @@
+/**
+ * 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.BytesColumnVector;
+
+/**
+ * Vectorized implementation of Bin(long) function that returns string.
+ */
+public class FuncBin extends FuncLongToString {
+  private static final long serialVersionUID = 1L;
+
+  FuncBin(int inputCol, int outputCol) {
+    super(inputCol, outputCol);
+  }
+
+  FuncBin() {
+    super();
+  }
+
+  @Override
+  void prepareResult(int i, long[] vector, BytesColumnVector outV) {
+    long num = vector[i];
+    // Extract the bits of num into bytes[] from right to left
+    int len = 0;
+    do {
+      len++;
+      bytes[bytes.length - len] = (byte) ('0' + (num & 1));
+      num >>>= 1;
+    } while (num != 0);
+    outV.setVal(i, bytes, bytes.length - len, len);
+  }
+}

Added: hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncHex.java
URL: http://svn.apache.org/viewvc/hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncHex.java?rev=1526183&view=auto
==============================================================================
--- hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncHex.java (added)
+++ hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncHex.java Wed Sep 25 14:13:37 2013
@@ -0,0 +1,48 @@
+/**
+ * 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.BytesColumnVector;
+
+// Vectorized implementation of Hex(long) that returns string
+public class FuncHex extends FuncLongToString {
+  private static final long serialVersionUID = 1L;
+
+  FuncHex(int inputCol, int outputCol) {
+    super(inputCol, outputCol);
+  }
+
+  FuncHex() {
+    super();
+  }
+
+  @Override
+  void prepareResult(int i, long[] vector, BytesColumnVector outV) {
+    long num = vector[i];
+    // Extract the bits of num into bytes[] from right to left
+    int len = 0;
+    do {
+      len++;
+      bytes[bytes.length - len] = (byte) Character.toUpperCase(Character
+          .forDigit((int) (num & 0xF), 16));
+      num >>>= 4;
+    } while (num != 0);
+    outV.setVal(i, bytes, bytes.length - len, len);
+  }
+}

Added: hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncLogWithBaseDoubleToDouble.java
URL: http://svn.apache.org/viewvc/hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncLogWithBaseDoubleToDouble.java?rev=1526183&view=auto
==============================================================================
--- hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncLogWithBaseDoubleToDouble.java (added)
+++ hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncLogWithBaseDoubleToDouble.java Wed Sep 25 14:13:37 2013
@@ -0,0 +1,47 @@
+/**
+ * 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 FuncLogWithBaseDoubleToDouble extends MathFuncDoubleToDouble {
+  private static final long serialVersionUID = 1L;
+
+  private double base;
+
+  FuncLogWithBaseDoubleToDouble(int colNum, double base, int outputColumn) {
+    super(colNum, outputColumn);
+    this.base = base;
+  }
+
+  FuncLogWithBaseDoubleToDouble() {
+    super();
+  }
+
+  @Override
+  double func(double d) {
+    return Math.log(d) / Math.log(base);
+  }
+
+  public double getBase() {
+    return base;
+  }
+
+  public void setBase(double base) {
+    this.base = base;
+  }
+}

Added: hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncLongToString.java
URL: http://svn.apache.org/viewvc/hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncLongToString.java?rev=1526183&view=auto
==============================================================================
--- hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncLongToString.java (added)
+++ hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncLongToString.java Wed Sep 25 14:13:37 2013
@@ -0,0 +1,144 @@
+/**
+ * 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.BytesColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+
+/**
+ * Superclass to support vectorized functions that take a long
+ * and return a string, optionally with additional configuraiton arguments.
+ * Used for bin(long), hex(long) etc.
+ */
+public abstract class FuncLongToString extends VectorExpression {
+  private static final long serialVersionUID = 1L;
+
+  private int inputCol;
+  private int outputCol;
+  protected transient byte[] bytes;
+
+  FuncLongToString(int inputCol, int outputCol) {
+    this.inputCol = inputCol;
+    this.outputCol = outputCol;
+    bytes = new byte[64];    // staging area for results, to avoid new() calls
+  }
+
+  FuncLongToString() {
+    bytes = new byte[64];
+  }
+
+  @Override
+  public void evaluate(VectorizedRowBatch batch) {
+
+    if (childExpressions != null) {
+      super.evaluateChildren(batch);
+    }
+
+    LongColumnVector inputColVector = (LongColumnVector) batch.cols[inputCol];
+    int[] sel = batch.selected;
+    int n = batch.size;
+    long[] vector = inputColVector.vector;
+    BytesColumnVector outV = (BytesColumnVector) batch.cols[outputCol];
+    outV.initBuffer();
+
+    if (n == 0) {
+      //Nothing to do
+      return;
+    }
+
+    if (inputColVector.noNulls) {
+      outV.noNulls = true;
+      if (inputColVector.isRepeating) {
+        outV.isRepeating = true;
+        prepareResult(0, vector, outV);
+      } else if (batch.selectedInUse) {
+        for(int j=0; j != n; j++) {
+          int i = sel[j];
+          prepareResult(i, vector, outV);
+        }
+        outV.isRepeating = false;
+      } else {
+        for(int i = 0; i != n; i++) {
+          prepareResult(i, vector, outV);
+        }
+        outV.isRepeating = false;
+      }
+    } else {
+      // Handle case with nulls. Don't do function if the value is null, to save time,
+      // because calling the function can be expensive.
+      outV.noNulls = false;
+      if (inputColVector.isRepeating) {
+        outV.isRepeating = true;
+        outV.isNull[0] = inputColVector.isNull[0];
+        if (!inputColVector.isNull[0]) {
+          prepareResult(0, vector, outV);
+        }
+      } else if (batch.selectedInUse) {
+        for(int j=0; j != n; j++) {
+          int i = sel[j];
+          if (!inputColVector.isNull[i]) {
+            prepareResult(i, vector, outV);
+          }
+          outV.isNull[i] = inputColVector.isNull[i];
+        }
+        outV.isRepeating = false;
+      } else {
+        for(int i = 0; i != n; i++) {
+          if (!inputColVector.isNull[i]) {
+            prepareResult(i, vector, outV);
+          }
+          outV.isNull[i] = inputColVector.isNull[i];
+        }
+        outV.isRepeating = false;
+      }
+    }
+  }
+
+  /* Evaluate result for position i (using bytes[] to avoid storage allocation costs)
+   * and set position i of the output vector to the result.
+   */
+  abstract void prepareResult(int i, long[] vector, BytesColumnVector outV);
+
+  @Override
+  public int getOutputColumn() {
+    return outputCol;
+  }
+
+  public int getOutputCol() {
+    return outputCol;
+  }
+
+  public void setOutputCol(int outputCol) {
+    this.outputCol = outputCol;
+  }
+
+  public int getInputCol() {
+    return inputCol;
+  }
+
+  public void setInputCol(int inputCol) {
+    this.inputCol = inputCol;
+  }
+
+  @Override
+  public String getOutputType() {
+    return "String";
+  }
+}

Added: hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncPowerDoubleToDouble.java
URL: http://svn.apache.org/viewvc/hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncPowerDoubleToDouble.java?rev=1526183&view=auto
==============================================================================
--- hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncPowerDoubleToDouble.java (added)
+++ hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncPowerDoubleToDouble.java Wed Sep 25 14:13:37 2013
@@ -0,0 +1,50 @@
+/**
+ * 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;
+
+/**
+ * Vectorized implementation for Pow(a, power) and Power(a, power)
+ */
+public class FuncPowerDoubleToDouble extends MathFuncDoubleToDouble {
+  private static final long serialVersionUID = 1L;
+
+  private double power;
+
+  FuncPowerDoubleToDouble(int colNum, double power, int outputColumn) {
+    super(colNum, outputColumn);
+    this.power = power;
+  }
+
+  FuncPowerDoubleToDouble() {
+    super();
+  }
+
+  @Override
+  public double func(double d) {
+    return Math.pow(d, power);
+  }
+
+  public double getPower() {
+    return power;
+  }
+
+  public void setPower(double power) {
+    this.power = power;
+  }
+}

Added: hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncRand.java
URL: http://svn.apache.org/viewvc/hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncRand.java?rev=1526183&view=auto
==============================================================================
--- hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncRand.java (added)
+++ hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncRand.java Wed Sep 25 14:13:37 2013
@@ -0,0 +1,108 @@
+/**
+ * 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.VectorizedRowBatch;
+import java.util.Random;
+
+/**
+ * Implements vectorized rand() and rand(seed) function evaluation.
+ */
+public class FuncRand extends VectorExpression {
+  private static final long serialVersionUID = 1L;
+
+  private int outputCol;
+  private Random random;
+
+  FuncRand(int outputCol) {
+    this.outputCol = outputCol;
+    random = null;
+  }
+
+  FuncRand(long seed, int outputCol) {
+    this.outputCol = outputCol;
+    random = new Random(seed);
+  }
+
+  FuncRand() {
+  }
+
+  @Override
+  public void evaluate(VectorizedRowBatch batch) {
+
+    if (childExpressions != null) {
+      this.evaluateChildren(batch);
+    }
+
+    DoubleColumnVector outputColVector = (DoubleColumnVector) batch.cols[outputCol];
+    int[] sel = batch.selected;
+    int n = batch.size;
+    double[] outputVector = outputColVector.vector;
+    outputColVector.noNulls = true;
+    outputColVector.isRepeating = false;
+
+    // return immediately if batch is empty
+    if (n == 0) {
+      return;
+    }
+
+    // For no-seed case, create new random number generator locally.
+    if (random == null) {
+      random = new Random();
+    }
+
+    if (batch.selectedInUse) {
+      for(int j = 0; j != n; j++) {
+        int i = sel[j];
+        outputVector[i] = random.nextDouble();
+      }
+    } else {
+      for(int i = 0; i != n; i++) {
+        outputVector[i] = random.nextDouble();
+      }
+    }
+ }
+
+  @Override
+  public int getOutputColumn() {
+    return outputCol;
+  }
+
+  public int getOutputCol() {
+    return outputCol;
+  }
+
+  public void setOutputCol(int outputCol) {
+    this.outputCol = outputCol;
+  }
+
+  public Random getRandom() {
+    return random;
+  }
+
+  public void setRandom(Random random) {
+    this.random = random;
+  }
+
+  @Override
+  public String getOutputType() {
+    return "double";
+  }
+}

Added: hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/MathExpr.java
URL: http://svn.apache.org/viewvc/hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/MathExpr.java?rev=1526183&view=auto
==============================================================================
--- hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/MathExpr.java (added)
+++ hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/MathExpr.java Wed Sep 25 14:13:37 2013
@@ -0,0 +1,51 @@
+/**
+ * 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;
+
+/** 
+ * 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) {
+      return (double) ((long) (d + 0.5d));
+    } else {
+      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;
+  }
+
+  public static double sign(long v) {
+    return v >= 0 ? 1.0 : -1.0;
+  }
+}

Added: hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/MathFuncDoubleToDouble.java
URL: http://svn.apache.org/viewvc/hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/MathFuncDoubleToDouble.java?rev=1526183&view=auto
==============================================================================
--- hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/MathFuncDoubleToDouble.java (added)
+++ hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/MathFuncDoubleToDouble.java Wed Sep 25 14:13:37 2013
@@ -0,0 +1,129 @@
+/**
+ * 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 lone 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 MathFuncDoubleToDouble 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.
+  abstract double func(double d);
+
+  MathFuncDoubleToDouble(int colNum, int outputColumn) {
+    this.colNum = colNum;
+    this.outputColumn = outputColumn;
+  }
+
+  MathFuncDoubleToDouble() {
+  }
+
+  @Override
+  public void evaluate(VectorizedRowBatch batch) {
+
+    if (childExpressions != null) {
+      this.evaluateChildren(batch);
+    }
+
+    DoubleColumnVector inputColVector = (DoubleColumnVector) 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;
+    double[] 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;
+    }
+  }
+
+  @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/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/RoundDoubleToDoubleWithNumDigits.java
URL: http://svn.apache.org/viewvc/hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/RoundDoubleToDoubleWithNumDigits.java?rev=1526183&view=auto
==============================================================================
--- hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/RoundDoubleToDoubleWithNumDigits.java (added)
+++ hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/RoundDoubleToDoubleWithNumDigits.java Wed Sep 25 14:13:37 2013
@@ -0,0 +1,68 @@
+/**
+ * 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 RoundDoubleToDoubleWithNumDigits extends MathFuncDoubleToDouble {
+  private static final long serialVersionUID = 1L;
+
+  private IntWritable decimalPlaces;
+  private UDFRound roundFunc;
+  private transient DoubleWritable dw;
+
+  RoundDoubleToDoubleWithNumDigits(int colNum, int decimalPlaces, int outputColumn) {
+    super(colNum, outputColumn);
+    this.decimalPlaces = new IntWritable();
+    this.decimalPlaces.set(decimalPlaces);
+    roundFunc = new UDFRound();
+    dw = new DoubleWritable();
+  }
+
+  RoundDoubleToDoubleWithNumDigits() {
+    super();
+    dw = new DoubleWritable();
+  }
+
+  // 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;
+  }
+}

Added: hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/StringConv.java
URL: http://svn.apache.org/viewvc/hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/StringConv.java?rev=1526183&view=auto
==============================================================================
--- hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/StringConv.java (added)
+++ hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/StringConv.java Wed Sep 25 14:13:37 2013
@@ -0,0 +1,65 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.ql.exec.vector.expressions;
+
+import org.apache.hadoop.hive.ql.exec.vector.expressions.StringUnaryUDF;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.hive.ql.udf.UDFConv;
+import org.apache.hadoop.io.IntWritable;
+
+
+/**
+ * Implement vectorized function conv(string, int,  int) returning string.
+ * Support for use on numbers instead of strings shall be implemented
+ * by inserting an explicit cast to string. There will not be VectorExpression
+ * classes specifically for conv applied to numbers.
+ */
+public class StringConv extends StringUnaryUDF {
+  private static final long serialVersionUID = 1L;
+
+  StringConv(int colNum, int outputColumn, int fromBase, int toBase) {
+    super(colNum, outputColumn, (IUDFUnaryString) new ConvWrapper(fromBase, toBase));
+  }
+
+  StringConv() {
+    super();
+  }
+
+  /* This wrapper class implements the evaluate() method expected
+   * by the superclass for use in the inner loop of vectorized expression
+   * evaluation. It holds the fromBase and toBase arguments to
+   * make the interface simply "Text evaluate(Text)" as expected.
+   */
+  static class ConvWrapper implements IUDFUnaryString {
+    UDFConv conv;
+    IntWritable fromBase;
+    IntWritable toBase;
+
+    ConvWrapper(int fromBase, int toBase) {
+      conv = new UDFConv();
+      this.fromBase = new IntWritable(fromBase);
+      this.toBase = new IntWritable(toBase);
+    }
+
+    @Override
+    public Text evaluate(Text s) {
+      return conv.evaluate(s, fromBase, toBase);
+    }
+  }
+}

Added: hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/StringHex.java
URL: http://svn.apache.org/viewvc/hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/StringHex.java?rev=1526183&view=auto
==============================================================================
--- hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/StringHex.java (added)
+++ hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/StringHex.java Wed Sep 25 14:13:37 2013
@@ -0,0 +1,29 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.ql.exec.vector.expressions;
+
+import org.apache.hadoop.hive.ql.exec.vector.expressions.StringUnaryUDF;
+import org.apache.hadoop.hive.ql.udf.UDFHex;
+
+// Implement vectorized function Hex(string) returning string
+public class StringHex extends StringUnaryUDF {
+  StringHex(int colNum, int outputColumn) {
+    super(colNum, outputColumn, (IUDFUnaryString) new UDFHex());
+  }
+}

Modified: hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/StringUnaryUDF.java
URL: http://svn.apache.org/viewvc/hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/StringUnaryUDF.java?rev=1526183&r1=1526182&r2=1526183&view=diff
==============================================================================
--- hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/StringUnaryUDF.java (original)
+++ hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/StringUnaryUDF.java Wed Sep 25 14:13:37 2013
@@ -18,6 +18,8 @@
 
 package org.apache.hadoop.hive.ql.exec.vector.expressions;
 
+import java.util.Arrays;
+
 import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.io.Text;
@@ -65,6 +67,7 @@ public class StringUnaryUDF extends Vect
     int [] start = inputColVector.start;
     int [] length = inputColVector.length;
     BytesColumnVector outV = (BytesColumnVector) batch.cols[outputColumn];
+    outV.initBuffer();
     Text t;
 
     if (n == 0) {
@@ -79,25 +82,33 @@ public class StringUnaryUDF extends Vect
     // existing built-in function.
 
     if (inputColVector.noNulls) {
-      outV.noNulls = true;
+      outV.noNulls = true; 
       if (inputColVector.isRepeating) {
         outV.isRepeating = true;
         s.set(vector[0], start[0], length[0]);
         t = func.evaluate(s);
-        outV.setRef(0, t.getBytes(), 0, t.getLength());
+        setString(outV, 0, t);
       } else if (batch.selectedInUse) {
-        for(int j=0; j != n; j++) {
+        for(int j = 0; j != n; j++) {
           int i = sel[j];
+          
+          /* Fill output isNull with false for selected elements since there is a chance we'll
+           * convert to noNulls == false in setString();
+           */     
+          outV.isNull[i] = false;
           s.set(vector[i], start[i], length[i]);
           t = func.evaluate(s);
-          outV.setRef(i, t.getBytes(), 0, t.getLength());
+          setString(outV, i, t);
         }
         outV.isRepeating = false;
       } else {
+        
+        // Set all elements to not null. The setString call can override this.
+        Arrays.fill(outV.isNull, 0, n - 1, false);
         for(int i = 0; i != n; i++) {
           s.set(vector[i], start[i], length[i]);
           t = func.evaluate(s);
-          outV.setRef(i, t.getBytes(), 0, t.getLength());
+          setString(outV, i, t);
         }
         outV.isRepeating = false;
       }
@@ -107,36 +118,50 @@ public class StringUnaryUDF extends Vect
       outV.noNulls = false;
       if (inputColVector.isRepeating) {
         outV.isRepeating = true;
-        outV.isNull[0] = inputColVector.isNull[0];
+        outV.isNull[0] = inputColVector.isNull[0]; // setString can override this
         if (!inputColVector.isNull[0]) {
           s.set(vector[0], start[0], length[0]);
           t = func.evaluate(s);
-          outV.setRef(0, t.getBytes(), 0, t.getLength());
+          setString(outV, 0, t);
         }
       } else if (batch.selectedInUse) {
-        for(int j=0; j != n; j++) {
+        for(int j = 0; j != n; j++) {
           int i = sel[j];
+          outV.isNull[i] = inputColVector.isNull[i]; // setString can override this          
           if (!inputColVector.isNull[i]) {
             s.set(vector[i], start[i], length[i]);
             t = func.evaluate(s);
-            outV.setRef(i, t.getBytes(), 0, t.getLength());
-          }
-          outV.isNull[i] = inputColVector.isNull[i];
+            setString(outV, i, t);
+          }  
         }
         outV.isRepeating = false;
       } else {
+        
+        // setString can override this null propagation
+        System.arraycopy(inputColVector.isNull, 0, outV.isNull, 0, n); 
         for(int i = 0; i != n; i++) {
           if (!inputColVector.isNull[i]) {
             s.set(vector[i], start[i], length[i]);
             t = func.evaluate(s);
-            outV.setRef(i, t.getBytes(), 0, t.getLength());
+            setString(outV, i, t);
           }
-          outV.isNull[i] = inputColVector.isNull[i];
         }
         outV.isRepeating = false;
       }
     }
   }
+  
+  /* Set the output string entry i to the contents of Text object t.
+   * If t is a null object reference, record that the value is a SQL NULL.
+   */
+  private static void setString(BytesColumnVector outV, int i, Text t) {
+    if (t == null) {
+      outV.noNulls = false;
+      outV.isNull[i] = true;
+      return;
+    }
+    outV.setVal(i, t.getBytes(), 0, t.getLength());
+  }
 
   @Override
   public int getOutputColumn() {

Added: hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/StringUnhex.java
URL: http://svn.apache.org/viewvc/hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/StringUnhex.java?rev=1526183&view=auto
==============================================================================
--- hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/StringUnhex.java (added)
+++ hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/StringUnhex.java Wed Sep 25 14:13:37 2013
@@ -0,0 +1,49 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.ql.exec.vector.expressions;
+
+import org.apache.hadoop.hive.ql.exec.vector.expressions.StringUnaryUDF;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.StringUnaryUDF.IUDFUnaryString;
+import org.apache.hadoop.hive.ql.udf.UDFUnhex;
+import org.apache.hadoop.io.Text;
+
+public class StringUnhex extends StringUnaryUDF {
+  private static final long serialVersionUID = 1L;
+
+  StringUnhex(int colNum, int outputColumn) {
+    super(colNum, outputColumn, new IUDFUnaryString() {
+
+      // Wrap the evaluate method of UDFUnhex to make it return the expected type, Text.
+      @Override
+      public Text evaluate(Text s) {
+        final UDFUnhex unhex = new UDFUnhex();
+        byte[] b = unhex.evaluate(s);
+        if (b == null) {
+          return null;
+        }
+        return new Text(b);
+      }
+
+    });
+  }
+
+  StringUnhex() {
+    super();
+  }
+}

Modified: hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFHex.java
URL: http://svn.apache.org/viewvc/hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFHex.java?rev=1526183&r1=1526182&r2=1526183&view=diff
==============================================================================
--- hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFHex.java (original)
+++ hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFHex.java Wed Sep 25 14:13:37 2013
@@ -20,6 +20,7 @@ package org.apache.hadoop.hive.ql.udf;
 
 import org.apache.hadoop.hive.ql.exec.Description;
 import org.apache.hadoop.hive.ql.exec.UDF;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.StringUnaryUDF.IUDFUnaryString;
 import org.apache.hadoop.io.BytesWritable;
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.LongWritable;
@@ -39,7 +40,7 @@ import org.apache.hadoop.io.Text;
     + "  'H1'\n"
     + "  > SELECT _FUNC_('Facebook') FROM src LIMIT 1;\n"
     + "  '46616365626F6F6B'")
-public class UDFHex extends UDF {
+public class UDFHex extends UDF implements IUDFUnaryString {
   private final Text result = new Text();
   private byte[] value = new byte[16];
 

Added: hive/branches/vectorization/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorMathFunctions.java
URL: http://svn.apache.org/viewvc/hive/branches/vectorization/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorMathFunctions.java?rev=1526183&view=auto
==============================================================================
--- hive/branches/vectorization/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorMathFunctions.java (added)
+++ hive/branches/vectorization/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorMathFunctions.java Wed Sep 25 14:13:37 2013
@@ -0,0 +1,569 @@
+/**
+ * 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 static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.io.UnsupportedEncodingException;
+import java.util.Arrays;
+
+import junit.framework.Assert;
+
+import org.apache.hadoop.hbase.client.coprocessor.Batch;
+import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.DoubleColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.*;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.*;
+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;
+  }
+
+  @Test
+  public void testVectorRound() {
+    VectorizedRowBatch b = getVectorizedRowBatchDoubleInDoubleOut();
+    VectorExpression expr = new FuncRoundDoubleToDouble(0, 1);
+    DoubleColumnVector resultV = (DoubleColumnVector) b.cols[1];
+    b.cols[0].noNulls = true;
+    expr.evaluate(b);
+    Assert.assertEquals(-2d, resultV.vector[0]);
+    Assert.assertEquals(-1d, resultV.vector[1]);
+    Assert.assertEquals(0d, resultV.vector[2]);
+    Assert.assertEquals(0d, resultV.vector[3]);
+    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;
+    resultV.noNulls = true;
+    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);   
+    
+    resultV.isRepeating = false;
+    b.cols[0].noNulls = true;
+    expr.evaluate(b);
+    Assert.assertEquals(-2d, resultV.vector[0]);
+    Assert.assertEquals(true, resultV.isRepeating);   
+  }
+  
+  @Test
+  public void testRoundToDecimalPlaces() {
+    VectorizedRowBatch b = getVectorizedRowBatchDoubleInDoubleOut();
+    VectorExpression expr = new RoundDoubleToDoubleWithNumDigits(0, 4, 1);
+    expr.evaluate(b);
+    DoubleColumnVector resultV = (DoubleColumnVector) b.cols[1];
+    
+    // Verify result is rounded to 4 digits
+    Assert.assertEquals(1.2346d, resultV.vector[7]);
+  }
+
+  public static VectorizedRowBatch getVectorizedRowBatchDoubleInLongOut() {
+    VectorizedRowBatch batch = new VectorizedRowBatch(2);
+    LongColumnVector lcv;
+    DoubleColumnVector dcv;
+    lcv = new LongColumnVector();
+    dcv = new DoubleColumnVector();
+    dcv.vector[0] = -1.5d;
+    dcv.vector[1] = -0.5d;
+    dcv.vector[2] = -0.1d;
+    dcv.vector[3] = 0d;
+    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;
+    DoubleColumnVector outV;
+    outV = new DoubleColumnVector();
+    inV = new DoubleColumnVector();
+    inV.vector[0] = -1.5d;
+    inV.vector[1] = -0.5d;
+    inV.vector[2] = -0.1d;
+    inV.vector[3] = 0d;
+    inV.vector[4] = 0.5d;
+    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;
+    DoubleColumnVector dcv;
+    lcv = new LongColumnVector();
+    dcv = new DoubleColumnVector();
+    lcv.vector[0] = -2;
+    lcv.vector[1] = -1;
+    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;
+    inV = new LongColumnVector();
+    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;
+    BytesColumnVector inS, outS;
+    inL = new LongColumnVector();
+    inS = new BytesColumnVector();
+    outS = new BytesColumnVector();
+    inL.vector[0] = 0;
+    inL.vector[1] = 255;
+    inL.vector[2] = 0;
+    inS.initBuffer();
+    try {
+      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);  
+    } catch (UnsupportedEncodingException e) {
+      e.printStackTrace();
+      Assert.assertTrue(false);
+    }
+
+    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
+   * propagation will be checked once for a single expansion of the template
+   * (for FuncRoundDoubleToDouble).
+   */
+  @Test
+  public void testVectorSin() {
+    VectorizedRowBatch b = getVectorizedRowBatchDoubleInDoubleOut();
+    DoubleColumnVector resultV = (DoubleColumnVector) b.cols[1];
+    b.cols[0].noNulls = true;
+    VectorExpression expr = new FuncSinDoubleToDouble(0, 1);
+    expr.evaluate(b);
+    Assert.assertEquals(Math.sin(0.5d), resultV.vector[4]);
+  }
+  
+  @Test
+  public void testVectorCos() {
+    VectorizedRowBatch b = getVectorizedRowBatchDoubleInDoubleOut();
+    DoubleColumnVector resultV = (DoubleColumnVector) b.cols[1];
+    b.cols[0].noNulls = true;
+    VectorExpression expr = new FuncCosDoubleToDouble(0, 1);
+    expr.evaluate(b);
+    Assert.assertEquals(Math.cos(0.5d), resultV.vector[4]);
+  }
+  
+  @Test
+  public void testVectorTan() {
+    VectorizedRowBatch b = getVectorizedRowBatchDoubleInDoubleOut();
+    DoubleColumnVector resultV = (DoubleColumnVector) b.cols[1];
+    b.cols[0].noNulls = true;
+    VectorExpression expr = new FuncTanDoubleToDouble(0, 1);
+    expr.evaluate(b);
+    Assert.assertEquals(Math.tan(0.5d), resultV.vector[4]);
+  }
+  
+  @Test
+  public void testVectorASin() {
+    VectorizedRowBatch b = getVectorizedRowBatchDoubleInDoubleOut();
+    DoubleColumnVector resultV = (DoubleColumnVector) b.cols[1];
+    b.cols[0].noNulls = true;
+    VectorExpression expr = new FuncASinDoubleToDouble(0, 1);
+    expr.evaluate(b);
+    Assert.assertEquals(Math.asin(0.5d), resultV.vector[4]);
+  }
+  
+  @Test
+  public void testVectorACos() {
+    VectorizedRowBatch b = getVectorizedRowBatchDoubleInDoubleOut();
+    DoubleColumnVector resultV = (DoubleColumnVector) b.cols[1];
+    b.cols[0].noNulls = true;
+    VectorExpression expr = new FuncACosDoubleToDouble(0, 1);
+    expr.evaluate(b);
+    Assert.assertEquals(Math.acos(0.5d), resultV.vector[4]);
+  }
+  
+  @Test
+  public void testVectorATan() {
+    VectorizedRowBatch b = getVectorizedRowBatchDoubleInDoubleOut();
+    DoubleColumnVector resultV = (DoubleColumnVector) b.cols[1];
+    b.cols[0].noNulls = true;
+    VectorExpression expr = new FuncATanDoubleToDouble(0, 1);
+    expr.evaluate(b);
+    Assert.assertEquals(Math.atan(0.5d), resultV.vector[4]);
+  }
+  
+  @Test
+  public void testVectorDegrees() {
+    VectorizedRowBatch b = getVectorizedRowBatchDoubleInDoubleOut();
+    DoubleColumnVector resultV = (DoubleColumnVector) b.cols[1];
+    b.cols[0].noNulls = true;
+    VectorExpression expr = new FuncDegreesDoubleToDouble(0, 1);
+    expr.evaluate(b);
+    Assert.assertEquals(Math.toDegrees(0.5d), resultV.vector[4]);
+  }
+  
+  @Test
+  public void testVectorRadians() {
+    VectorizedRowBatch b = getVectorizedRowBatchDoubleInDoubleOut();
+    DoubleColumnVector resultV = (DoubleColumnVector) b.cols[1];
+    b.cols[0].noNulls = true;
+    VectorExpression expr = new FuncRadiansDoubleToDouble(0, 1);
+    expr.evaluate(b);
+    Assert.assertEquals(Math.toRadians(0.5d), resultV.vector[4]);
+  }
+  
+  @Test
+  public void testVectorFloor() {
+    VectorizedRowBatch b = getVectorizedRowBatchDoubleInLongOut();
+    LongColumnVector resultV = (LongColumnVector) b.cols[1];
+    b.cols[0].noNulls = true;
+    VectorExpression expr = new FuncFloorDoubleToLong(0, 1);
+    expr.evaluate(b);
+    Assert.assertEquals(-2, resultV.vector[0]);
+    Assert.assertEquals(1, resultV.vector[6]);
+  }
+  
+  @Test
+  public void testVectorCeil() {
+    VectorizedRowBatch b = getVectorizedRowBatchDoubleInLongOut();
+    LongColumnVector resultV = (LongColumnVector) b.cols[1];
+    b.cols[0].noNulls = true;
+    VectorExpression expr = new FuncCeilDoubleToLong(0, 1);
+    expr.evaluate(b);
+    Assert.assertEquals(-1, resultV.vector[0]);
+    Assert.assertEquals(2, resultV.vector[6]);
+  }
+  
+  @Test
+  public void testVectorExp() {
+    VectorizedRowBatch b = getVectorizedRowBatchDoubleInDoubleOut();
+    DoubleColumnVector resultV = (DoubleColumnVector) b.cols[1];
+    b.cols[0].noNulls = true;
+    VectorExpression expr = new FuncExpDoubleToDouble(0, 1);
+    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];
+    b.cols[0].noNulls = true;
+    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];
+    b.cols[0].noNulls = true;
+    expr = new FuncLnLongToDouble(0, 1);
+    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];
+    b.cols[0].noNulls = true;
+    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];
+    b.cols[0].noNulls = true;
+    expr = new FuncLog2LongToDouble(0, 1);
+    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];
+    b.cols[0].noNulls = true;
+    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];
+    b.cols[0].noNulls = true;
+    expr = new FuncLog10LongToDouble(0, 1);
+    expr.evaluate(b);
+    Assert.assertEquals(Math.log(1) / Math.log(10), resultV.vector[3]);
+  }
+  
+  @Test
+  public void testVectorRand() {
+    VectorizedRowBatch b = new VectorizedRowBatch(1);
+    DoubleColumnVector v = new DoubleColumnVector();
+    b.cols[0] = v;
+    b.size = VectorizedRowBatch.DEFAULT_SIZE;
+    int n = b.size;
+    v.noNulls = true;
+    VectorExpression expr = new FuncRand(0);
+    expr.evaluate(b);
+    double sum = 0;
+    for(int i = 0; i != n; i++) {
+      sum += v.vector[i];
+      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 
+     * 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);
+    expr.evaluate(b);
+    sum = 0;
+    for(int i = 0; i != n; i++) {
+      sum += v.vector[i];
+      Assert.assertTrue(v.vector[i] >= 0.0 && v.vector[i] <= 1.0);
+    }
+    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);
+    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);
+    expr.evaluate(b);
+    Assert.assertTrue(equalsWithinTolerance(0.5d * 0.5d, resultV.vector[4]));
+  }
+  
+  @Test
+  public void testVectorSqrt() {
+    VectorizedRowBatch b = getVectorizedRowBatchDoubleInDoubleOut();
+    DoubleColumnVector resultV = (DoubleColumnVector) b.cols[1];
+    b.cols[0].noNulls = true;
+    VectorExpression expr = new FuncSqrtDoubleToDouble(0, 1);
+    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];
+    b.cols[0].noNulls = true;
+    VectorExpression expr = new FuncAbsDoubleToDouble(0, 1);
+    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];
+    b.cols[0].noNulls = true;
+    expr = new FuncAbsLongToLong(0, 1);
+    expr.evaluate(b);
+    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];
+    b.cols[0].noNulls = true;
+    VectorExpression expr = new FuncSignDoubleToDouble(0, 1);
+    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];
+    b.cols[0].noNulls = true;
+    expr = new FuncSignLongToDouble(0, 1);
+    expr.evaluate(b);
+    Assert.assertEquals(-1.0d, resultV.vector[0]);
+    Assert.assertEquals(1.0d, resultV.vector[4]);   
+  }
+  
+  @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); 
+    expr.evaluate(b);
+    String s = new String(resultV.vector[1], resultV.start[1], resultV.length[1]);
+    Assert.assertEquals("11111111", s);
+  }
+  
+  @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); 
+    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.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.
+  }
+}

Modified: hive/branches/vectorization/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorStringExpressions.java
URL: http://svn.apache.org/viewvc/hive/branches/vectorization/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorStringExpressions.java?rev=1526183&r1=1526182&r2=1526183&view=diff
==============================================================================
--- hive/branches/vectorization/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorStringExpressions.java (original)
+++ hive/branches/vectorization/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorStringExpressions.java Wed Sep 25 14:13:37 2013
@@ -391,6 +391,7 @@ public class TestVectorStringExpressions
     BytesColumnVector v = new BytesColumnVector(VectorizedRowBatch.DEFAULT_SIZE);
     batch.cols[0] = v;
     BytesColumnVector outV = new BytesColumnVector(VectorizedRowBatch.DEFAULT_SIZE);
+    outV.initBuffer();
     batch.cols[1] = outV;
     /*
      * Add these 3 values: