You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by zs...@apache.org on 2010/01/21 11:38:15 UTC

svn commit: r901644 [29/37] - in /hadoop/hive/trunk: ./ ql/src/java/org/apache/hadoop/hive/ql/ ql/src/java/org/apache/hadoop/hive/ql/exec/ ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/ ql/src/java/org/apache/hadoop/hive/ql/history/ ql/src/jav...

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFVariance.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFVariance.java?rev=901644&r1=901643&r2=901644&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFVariance.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFVariance.java Thu Jan 21 10:37:58 2010
@@ -41,73 +41,63 @@
 import org.apache.hadoop.util.StringUtils;
 
 /**
- * Compute the variance. This class is extended by:
- *   GenericUDAFVarianceSample
- *   GenericUDAFStd
- *   GenericUDAFStdSample
- *
+ * Compute the variance. This class is extended by: GenericUDAFVarianceSample
+ * GenericUDAFStd GenericUDAFStdSample
+ * 
  */
-@description(
-    name = "variance,var_pop",
-    value = "_FUNC_(x) - Returns the variance of a set of numbers"
-)
+@description(name = "variance,var_pop", value = "_FUNC_(x) - Returns the variance of a set of numbers")
 public class GenericUDAFVariance implements GenericUDAFResolver {
-  
+
   static final Log LOG = LogFactory.getLog(GenericUDAFVariance.class.getName());
-  
+
   @Override
-  public GenericUDAFEvaluator getEvaluator(
-      TypeInfo[] parameters) throws SemanticException {
+  public GenericUDAFEvaluator getEvaluator(TypeInfo[] parameters)
+      throws SemanticException {
     if (parameters.length != 1) {
       throw new UDFArgumentTypeException(parameters.length - 1,
           "Exactly one argument is expected.");
     }
-    
+
     if (parameters[0].getCategory() != ObjectInspector.Category.PRIMITIVE) {
       throw new UDFArgumentTypeException(0,
-          "Only primitive type arguments are accepted but " 
-          + parameters[0].getTypeName() + " is passed.");
+          "Only primitive type arguments are accepted but "
+              + parameters[0].getTypeName() + " is passed.");
     }
-    switch (((PrimitiveTypeInfo)parameters[0]).getPrimitiveCategory()) {
-      case BYTE:
-      case SHORT:
-      case INT:
-      case LONG:
-      case FLOAT:
-      case DOUBLE:
-      case STRING:
-        return new GenericUDAFVarianceEvaluator();
-      case BOOLEAN:
-      default:
-        throw new UDFArgumentTypeException(0,
-            "Only numeric or string type arguments are accepted but " 
-            + parameters[0].getTypeName() + " is passed.");
+    switch (((PrimitiveTypeInfo) parameters[0]).getPrimitiveCategory()) {
+    case BYTE:
+    case SHORT:
+    case INT:
+    case LONG:
+    case FLOAT:
+    case DOUBLE:
+    case STRING:
+      return new GenericUDAFVarianceEvaluator();
+    case BOOLEAN:
+    default:
+      throw new UDFArgumentTypeException(0,
+          "Only numeric or string type arguments are accepted but "
+              + parameters[0].getTypeName() + " is passed.");
     }
   }
-  
+
   /**
    * Evaluate the variance using the following modification of the formula from
    * The Art of Computer Programming, vol. 2, p. 232:
-   *   
-   *   variance = variance1 + variance2 + n*alpha^2 + m*betha^2
-   *   
-   * where:
-   *   - variance is sum[x-avg^2] (this is actually n times the variance) and is
-   *   updated at every step.
-   *   - n is the count of elements in chunk1
-   *   - m is the count of elements in chunk2
-   *   - alpha = avg-a
-   *   - betha = avg-b
-   *   - avg is the the average of all elements from both chunks
-   *   - a is the average of elements in chunk1
-   *   - b is the average of elements in chunk2
+   * 
+   * variance = variance1 + variance2 + n*alpha^2 + m*betha^2
+   * 
+   * where: - variance is sum[x-avg^2] (this is actually n times the variance)
+   * and is updated at every step. - n is the count of elements in chunk1 - m is
+   * the count of elements in chunk2 - alpha = avg-a - betha = avg-b - avg is
+   * the the average of all elements from both chunks - a is the average of
+   * elements in chunk1 - b is the average of elements in chunk2
    * 
    */
   public static class GenericUDAFVarianceEvaluator extends GenericUDAFEvaluator {
-    
+
     // For PARTIAL1 and COMPLETE
     PrimitiveObjectInspector inputOI;
-    
+
     // For PARTIAL2 and FINAL
     StructObjectInspector soi;
     StructField countField;
@@ -116,60 +106,60 @@
     LongObjectInspector countFieldOI;
     DoubleObjectInspector sumFieldOI;
     DoubleObjectInspector varianceFieldOI;
-    
+
     // For PARTIAL1 and PARTIAL2
     Object[] partialResult;
-    
+
     // For FINAL and COMPLETE
     DoubleWritable result;
-    
+
     @Override
     public ObjectInspector init(Mode m, ObjectInspector[] parameters)
         throws HiveException {
-      assert(parameters.length == 1);
+      assert (parameters.length == 1);
       super.init(m, parameters);
-      
+
       // init input
-      if (mode == mode.PARTIAL1 || mode == mode.COMPLETE) {
-        inputOI = (PrimitiveObjectInspector)parameters[0];
+      if (mode == Mode.PARTIAL1 || mode == Mode.COMPLETE) {
+        inputOI = (PrimitiveObjectInspector) parameters[0];
       } else {
-        soi = (StructObjectInspector)parameters[0];
-        
+        soi = (StructObjectInspector) parameters[0];
+
         countField = soi.getStructFieldRef("count");
         sumField = soi.getStructFieldRef("sum");
         varianceField = soi.getStructFieldRef("variance");
-        
-        countFieldOI = 
-          (LongObjectInspector)countField.getFieldObjectInspector();
-        sumFieldOI = (DoubleObjectInspector)sumField.getFieldObjectInspector();
-        varianceFieldOI = 
-          (DoubleObjectInspector)varianceField.getFieldObjectInspector();
+
+        countFieldOI = (LongObjectInspector) countField
+            .getFieldObjectInspector();
+        sumFieldOI = (DoubleObjectInspector) sumField.getFieldObjectInspector();
+        varianceFieldOI = (DoubleObjectInspector) varianceField
+            .getFieldObjectInspector();
       }
-      
+
       // init output
-      if (mode == mode.PARTIAL1 || mode == mode.PARTIAL2) {
+      if (mode == Mode.PARTIAL1 || mode == Mode.PARTIAL2) {
         // The output of a partial aggregation is a struct containing
-        // a long count and doubles sum and variance. 
-        
+        // a long count and doubles sum and variance.
+
         ArrayList<ObjectInspector> foi = new ArrayList<ObjectInspector>();
-        
+
         foi.add(PrimitiveObjectInspectorFactory.writableLongObjectInspector);
         foi.add(PrimitiveObjectInspectorFactory.writableDoubleObjectInspector);
         foi.add(PrimitiveObjectInspectorFactory.writableDoubleObjectInspector);
-        
+
         ArrayList<String> fname = new ArrayList<String>();
         fname.add("count");
         fname.add("sum");
         fname.add("variance");
-        
+
         partialResult = new Object[3];
         partialResult[0] = new LongWritable(0);
         partialResult[1] = new DoubleWritable(0);
         partialResult[2] = new DoubleWritable(0);
-        
-        return ObjectInspectorFactory.getStandardStructObjectInspector(
-            fname, foi);
-        
+
+        return ObjectInspectorFactory.getStandardStructObjectInspector(fname,
+            foi);
+
       } else {
         result = new DoubleWritable(0);
         return PrimitiveObjectInspectorFactory.writableDoubleObjectInspector;
@@ -191,42 +181,44 @@
 
     @Override
     public void reset(AggregationBuffer agg) throws HiveException {
-      StdAgg myagg = (StdAgg)agg;
+      StdAgg myagg = (StdAgg) agg;
       myagg.count = 0;
-      myagg.sum = 0;     
+      myagg.sum = 0;
       myagg.variance = 0;
     }
-    
+
     boolean warned = false;
-    
+
     @Override
-    public void iterate(AggregationBuffer agg, Object[] parameters) 
-    throws HiveException {
-      assert(parameters.length == 1);
+    public void iterate(AggregationBuffer agg, Object[] parameters)
+        throws HiveException {
+      assert (parameters.length == 1);
       Object p = parameters[0];
       if (p != null) {
-        StdAgg myagg = (StdAgg)agg;
+        StdAgg myagg = (StdAgg) agg;
         try {
-          double v = PrimitiveObjectInspectorUtils.getDouble(p, 
-              (PrimitiveObjectInspector)inputOI);
-         
-          if(myagg.count != 0) { // if count==0 => the variance is going to be 0
-                                // after 1 iteration
-            double alpha = (myagg.sum + v) / (myagg.count+1) 
-                            - myagg.sum / myagg.count;
-            double betha = (myagg.sum + v) / (myagg.count+1) - v;
-            
+          double v = PrimitiveObjectInspectorUtils.getDouble(p, inputOI);
+
+          if (myagg.count != 0) { // if count==0 => the variance is going to be
+                                  // 0
+            // after 1 iteration
+            double alpha = (myagg.sum + v) / (myagg.count + 1) - myagg.sum
+                / myagg.count;
+            double betha = (myagg.sum + v) / (myagg.count + 1) - v;
+
             // variance = variance1 + variance2 + n*alpha^2 + m*betha^2
             // => variance += n*alpha^2 + betha^2
-            myagg.variance += myagg.count*alpha*alpha + betha*betha;
+            myagg.variance += myagg.count * alpha * alpha + betha * betha;
           }
           myagg.count++;
           myagg.sum += v;
         } catch (NumberFormatException e) {
           if (!warned) {
             warned = true;
-            LOG.warn(getClass().getSimpleName() + " " + StringUtils.stringifyException(e));
-            LOG.warn(getClass().getSimpleName() + " ignoring similar exceptions.");
+            LOG.warn(getClass().getSimpleName() + " "
+                + StringUtils.stringifyException(e));
+            LOG.warn(getClass().getSimpleName()
+                + " ignoring similar exceptions.");
           }
         }
       }
@@ -234,60 +226,61 @@
 
     @Override
     public Object terminatePartial(AggregationBuffer agg) throws HiveException {
-      StdAgg myagg = (StdAgg)agg;
-      ((LongWritable)partialResult[0]).set(myagg.count);
-      ((DoubleWritable)partialResult[1]).set(myagg.sum);
-      ((DoubleWritable)partialResult[2]).set(myagg.variance);
+      StdAgg myagg = (StdAgg) agg;
+      ((LongWritable) partialResult[0]).set(myagg.count);
+      ((DoubleWritable) partialResult[1]).set(myagg.sum);
+      ((DoubleWritable) partialResult[2]).set(myagg.variance);
       return partialResult;
     }
 
     @Override
-    public void merge(AggregationBuffer agg, Object partial) throws HiveException {
+    public void merge(AggregationBuffer agg, Object partial)
+        throws HiveException {
       if (partial != null) {
-        StdAgg myagg = (StdAgg)agg;
-        
+        StdAgg myagg = (StdAgg) agg;
+
         Object partialCount = soi.getStructFieldData(partial, countField);
         Object partialSum = soi.getStructFieldData(partial, sumField);
         Object partialVariance = soi.getStructFieldData(partial, varianceField);
-        
+
         long n = myagg.count;
         long m = countFieldOI.get(partialCount);
-        
-        if(n == 0) {
+
+        if (n == 0) {
           // Just copy the information since there is nothing so far
           myagg.variance = sumFieldOI.get(partialVariance);
           myagg.count = countFieldOI.get(partialCount);
           myagg.sum = sumFieldOI.get(partialSum);
         }
-        
-        if(m != 0 && n != 0) {
+
+        if (m != 0 && n != 0) {
           // Merge the two partials
-          
+
           double a = myagg.sum;
           double b = sumFieldOI.get(partialSum);
-          
-          double alpha = (a+b)/(n+m) - a/n;
-          double betha = (a+b)/(n+m) - b/m;
-          
+
+          double alpha = (a + b) / (n + m) - a / n;
+          double betha = (a + b) / (n + m) - b / m;
+
           // variance = variance1 + variance2 + n*alpha^2 + m*betha^2
           myagg.variance += sumFieldOI.get(partialVariance)
-                            + (n*alpha*alpha + m*betha*betha);
+              + (n * alpha * alpha + m * betha * betha);
           myagg.count += m;
           myagg.sum += b;
         }
-        
+
       }
     }
 
     @Override
     public Object terminate(AggregationBuffer agg) throws HiveException {
-      StdAgg myagg = (StdAgg)agg;
-      
+      StdAgg myagg = (StdAgg) agg;
+
       if (myagg.count == 0) { // SQL standard - return null for zero elements
         return null;
       } else {
-        if(myagg.count > 1) { 
-          result.set(myagg.variance / (myagg.count)); 
+        if (myagg.count > 1) {
+          result.set(myagg.variance / (myagg.count));
         } else { // for one element the variance is always 0
           result.set(0);
         }

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFVarianceSample.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFVarianceSample.java?rev=901644&r1=901643&r2=901644&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFVarianceSample.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFVarianceSample.java Thu Jan 21 10:37:58 2010
@@ -26,62 +26,59 @@
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
 
 /**
- * Compute the sample variance by extending GenericUDAFVariance and 
- * overriding the terminate() method of the evaluator.
- *
+ * Compute the sample variance by extending GenericUDAFVariance and overriding
+ * the terminate() method of the evaluator.
+ * 
  */
-@description(
-    name = "var_samp",
-    value = "_FUNC_(x) - Returns the sample variance of a set of numbers"
-)
+@description(name = "var_samp", value = "_FUNC_(x) - Returns the sample variance of a set of numbers")
 public class GenericUDAFVarianceSample extends GenericUDAFVariance {
-  
+
   @Override
-  public GenericUDAFEvaluator getEvaluator(
-      TypeInfo[] parameters) throws SemanticException {
+  public GenericUDAFEvaluator getEvaluator(TypeInfo[] parameters)
+      throws SemanticException {
     if (parameters.length != 1) {
       throw new UDFArgumentTypeException(parameters.length - 1,
           "Exactly one argument is expected.");
     }
-    
+
     if (parameters[0].getCategory() != ObjectInspector.Category.PRIMITIVE) {
       throw new UDFArgumentTypeException(0,
-          "Only primitive type arguments are accepted but " 
-          + parameters[0].getTypeName() + " is passed.");
+          "Only primitive type arguments are accepted but "
+              + parameters[0].getTypeName() + " is passed.");
     }
-    switch (((PrimitiveTypeInfo)parameters[0]).getPrimitiveCategory()) {
-      case BYTE:
-      case SHORT:
-      case INT:
-      case LONG:
-      case FLOAT:
-      case DOUBLE:
-      case STRING:
-        return new GenericUDAFVarianceSampleEvaluator();
-      case BOOLEAN:
-      default:
-        throw new UDFArgumentTypeException(0,
-            "Only numeric or string type arguments are accepted but " 
-            + parameters[0].getTypeName() + " is passed.");
+    switch (((PrimitiveTypeInfo) parameters[0]).getPrimitiveCategory()) {
+    case BYTE:
+    case SHORT:
+    case INT:
+    case LONG:
+    case FLOAT:
+    case DOUBLE:
+    case STRING:
+      return new GenericUDAFVarianceSampleEvaluator();
+    case BOOLEAN:
+    default:
+      throw new UDFArgumentTypeException(0,
+          "Only numeric or string type arguments are accepted but "
+              + parameters[0].getTypeName() + " is passed.");
     }
   }
-  
+
   /**
-   * Compute the sample variance by extending 
-   * GenericUDAFVarianceEvaluator and overriding the terminate() method of the
-   * evaluator 
+   * Compute the sample variance by extending GenericUDAFVarianceEvaluator and
+   * overriding the terminate() method of the evaluator
    */
-  public static class GenericUDAFVarianceSampleEvaluator extends GenericUDAFVarianceEvaluator {
+  public static class GenericUDAFVarianceSampleEvaluator extends
+      GenericUDAFVarianceEvaluator {
 
     @Override
     public Object terminate(AggregationBuffer agg) throws HiveException {
-      StdAgg myagg = (StdAgg)agg;
-      
+      StdAgg myagg = (StdAgg) agg;
+
       if (myagg.count == 0) { // SQL standard - return null for zero elements
         return null;
       } else {
-        if(myagg.count > 1) { 
-          result.set(myagg.variance / (myagg.count-1)); 
+        if (myagg.count > 1) {
+          result.set(myagg.variance / (myagg.count - 1));
         } else { // for one element the variance is always 0
           result.set(0);
         }

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDF.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDF.java?rev=901644&r1=901643&r2=901644&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDF.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDF.java Thu Jan 21 10:37:58 2010
@@ -28,26 +28,24 @@
  * 
  * New GenericUDF classes need to inherit from this GenericUDF class.
  * 
- * The GenericUDF are superior to normal UDFs in the following ways:
- * 1. It can accept arguments of complex types, and return complex types.
- * 2. It can accept variable length of arguments.
- * 3. It can accept an infinite number of function signature - for example, 
- *    it's easy to write a GenericUDF that accepts array<int>, 
- *    array<array<int>> and so on (arbitrary levels of nesting).
- * 4. It can do short-circuit evaluations using DeferedObject.  
+ * The GenericUDF are superior to normal UDFs in the following ways: 1. It can
+ * accept arguments of complex types, and return complex types. 2. It can accept
+ * variable length of arguments. 3. It can accept an infinite number of function
+ * signature - for example, it's easy to write a GenericUDF that accepts
+ * array<int>, array<array<int>> and so on (arbitrary levels of nesting). 4. It
+ * can do short-circuit evaluations using DeferedObject.
  */
-@UDFType(deterministic=true)
+@UDFType(deterministic = true)
 public abstract class GenericUDF {
-  
+
   /**
-   * A Defered Object allows us to do lazy-evaluation
-   * and short-circuiting.
+   * A Defered Object allows us to do lazy-evaluation and short-circuiting.
    * GenericUDF use DeferedObject to pass arguments.
    */
   public static interface DeferredObject {
-    public Object get() throws HiveException; 
+    public Object get() throws HiveException;
   };
-  
+
   /**
    * The constructor
    */
@@ -58,26 +56,30 @@
    * Initialize this GenericUDF. This will be called once and only once per
    * GenericUDF instance.
    * 
-   * @param arguments     The ObjectInspector for the arguments
+   * @param arguments
+   *          The ObjectInspector for the arguments
    * @throws UDFArgumentException
-   *                      Thrown when arguments have wrong types, wrong length, etc.
-   * @return              The ObjectInspector for the return value
+   *           Thrown when arguments have wrong types, wrong length, etc.
+   * @return The ObjectInspector for the return value
    */
-  public abstract ObjectInspector initialize(ObjectInspector[] arguments) 
+  public abstract ObjectInspector initialize(ObjectInspector[] arguments)
       throws UDFArgumentException;
-  
+
   /**
    * Evaluate the GenericUDF with the arguments.
-   * @param arguments  The arguments as DeferedObject, use DeferedObject.get() to
-   *                   get the actual argument Object.  The Objects can be inspected
-   *                   by the ObjectInspectors passed in the initialize call.
-   * @return The 
+   * 
+   * @param arguments
+   *          The arguments as DeferedObject, use DeferedObject.get() to get the
+   *          actual argument Object. The Objects can be inspected by the
+   *          ObjectInspectors passed in the initialize call.
+   * @return The
    */
-  public abstract Object evaluate(DeferredObject[] arguments) throws HiveException;
-  
+  public abstract Object evaluate(DeferredObject[] arguments)
+      throws HiveException;
+
   /**
    * Get the String to be displayed in explain.
    */
   public abstract String getDisplayString(String[] children);
-  
+
 }

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFArray.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFArray.java?rev=901644&r1=901643&r2=901644&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFArray.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFArray.java Thu Jan 21 10:37:58 2010
@@ -21,7 +21,6 @@
 import java.util.ArrayList;
 
 import org.apache.hadoop.hive.ql.exec.UDFArgumentException;
-import org.apache.hadoop.hive.ql.exec.UDFArgumentLengthException;
 import org.apache.hadoop.hive.ql.exec.UDFArgumentTypeException;
 import org.apache.hadoop.hive.ql.exec.description;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
@@ -32,50 +31,48 @@
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorConverters.Converter;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
 
-
-@description(
-    name = "array",
-    value = "_FUNC_(n0, n1...) - Creates an array with the given elements "
-)
-
+@description(name = "array", value = "_FUNC_(n0, n1...) - Creates an array with the given elements ")
 public class GenericUDFArray extends GenericUDF {
-  
+
   Converter[] converters;
   ArrayList<Object> ret = new ArrayList<Object>();
+
   @Override
   public ObjectInspector initialize(ObjectInspector[] arguments)
-  throws UDFArgumentException {
+      throws UDFArgumentException {
 
     GenericUDFUtils.ReturnObjectInspectorResolver returnOIResolver;
 
     returnOIResolver = new GenericUDFUtils.ReturnObjectInspectorResolver(true);
 
-    for(int i=0; i<arguments.length; i++) {
-      if(!returnOIResolver.update(arguments[i])) {
-        throw new UDFArgumentTypeException(i, "Argument type \"" + arguments[i].getTypeName() + 
-            "\" is different from preceding arguments. " + 
-            "Previous type was \"" + arguments[i-1].getTypeName() + "\"");
+    for (int i = 0; i < arguments.length; i++) {
+      if (!returnOIResolver.update(arguments[i])) {
+        throw new UDFArgumentTypeException(i, "Argument type \""
+            + arguments[i].getTypeName()
+            + "\" is different from preceding arguments. "
+            + "Previous type was \"" + arguments[i - 1].getTypeName() + "\"");
       }
     }
-    
+
     converters = new Converter[arguments.length];
 
     ObjectInspector returnOI = returnOIResolver.get();
-    if(returnOI == null) {
-      returnOI = PrimitiveObjectInspectorFactory.getPrimitiveJavaObjectInspector(PrimitiveObjectInspector.PrimitiveCategory.STRING);
+    if (returnOI == null) {
+      returnOI = PrimitiveObjectInspectorFactory
+          .getPrimitiveJavaObjectInspector(PrimitiveObjectInspector.PrimitiveCategory.STRING);
     }
-    for(int i=0; i<arguments.length; i++) {
-      converters[i] = ObjectInspectorConverters.getConverter(arguments[i], returnOI);
+    for (int i = 0; i < arguments.length; i++) {
+      converters[i] = ObjectInspectorConverters.getConverter(arguments[i],
+          returnOI);
     }
-    
-    return ObjectInspectorFactory.getStandardListObjectInspector(
-        returnOI);
+
+    return ObjectInspectorFactory.getStandardListObjectInspector(returnOI);
   }
 
   @Override
   public Object evaluate(DeferredObject[] arguments) throws HiveException {
     ret.clear();
-    for(int i=0; i<arguments.length; i++) {
+    for (int i = 0; i < arguments.length; i++) {
       ret.add(converters[i].convert(arguments[i].get()));
     }
     return ret;
@@ -86,9 +83,9 @@
   public String getDisplayString(String[] children) {
     StringBuilder sb = new StringBuilder();
     sb.append("array(");
-    for(int i=0; i<children.length; i++) {
+    for (int i = 0; i < children.length; i++) {
       sb.append(children[i]);
-      if(i+1 != children.length) {
+      if (i + 1 != children.length) {
         sb.append(",");
       }
     }

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFBridge.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFBridge.java?rev=901644&r1=901643&r2=901644&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFBridge.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFBridge.java Thu Jan 21 10:37:58 2010
@@ -24,7 +24,6 @@
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hive.ql.exec.AmbiguousMethodException;
 import org.apache.hadoop.hive.ql.exec.FunctionRegistry;
 import org.apache.hadoop.hive.ql.exec.UDF;
 import org.apache.hadoop.hive.ql.exec.UDFArgumentException;
@@ -41,8 +40,8 @@
  * GenericUDFBridge encapsulates UDF to provide the same interface as
  * GenericUDF.
  * 
- * Note that GenericUDFBridge implements Serializable because the name of
- * the UDF class needs to be serialized with the plan.
+ * Note that GenericUDFBridge implements Serializable because the name of the
+ * UDF class needs to be serialized with the plan.
  * 
  */
 public class GenericUDFBridge extends GenericUDF implements Serializable {
@@ -53,25 +52,28 @@
    * The name of the UDF.
    */
   String udfName;
-  
+
   /**
-   * Whether the UDF is an operator or not.
-   * This controls how the display string is generated. 
+   * Whether the UDF is an operator or not. This controls how the display string
+   * is generated.
    */
   boolean isOperator;
-  
+
   /**
    * The underlying UDF class.
    */
   Class<? extends UDF> udfClass;
-  
+
   /**
    * Greate a new GenericUDFBridge object.
-   * @param udfName     The name of the corresponding udf.
+   * 
+   * @param udfName
+   *          The name of the corresponding udf.
    * @param isOperator
    * @param udfClass
    */
-  public GenericUDFBridge(String udfName, boolean isOperator, Class<? extends UDF> udfClass) {
+  public GenericUDFBridge(String udfName, boolean isOperator,
+      Class<? extends UDF> udfClass) {
     this.udfName = udfName;
     this.isOperator = isOperator;
     this.udfClass = udfClass;
@@ -80,15 +82,15 @@
   // For Java serialization only
   public GenericUDFBridge() {
   }
-  
+
   public void setUdfName(String udfName) {
     this.udfName = udfName;
   }
-  
+
   public String getUdfName() {
     return udfName;
   }
-  
+
   public boolean isOperator() {
     return isOperator;
   }
@@ -100,18 +102,18 @@
   public void setUdfClass(Class<? extends UDF> udfClass) {
     this.udfClass = udfClass;
   }
-  
+
   public Class<? extends UDF> getUdfClass() {
     return udfClass;
   }
-  
+
   /**
    * The underlying method of the UDF class.
    */
   transient Method udfMethod;
-  
+
   /**
-   * Helper to convert the parameters before passing to udfMethod.  
+   * Helper to convert the parameters before passing to udfMethod.
    */
   transient ConversionHelper conversionHelper;
   /**
@@ -122,46 +124,49 @@
    * The non-deferred real arguments for method invocation
    */
   transient Object[] realArguments;
-  
+
   @Override
   public ObjectInspector initialize(ObjectInspector[] arguments)
       throws UDFArgumentException {
-    
-    udf = (UDF)ReflectionUtils.newInstance(udfClass, null);
-    
+
+    udf = (UDF) ReflectionUtils.newInstance(udfClass, null);
+
     // Resolve for the method based on argument types
-    ArrayList<TypeInfo> argumentTypeInfos = new ArrayList<TypeInfo>(arguments.length);
-    for (int i=0; i<arguments.length; i++) {
-      argumentTypeInfos.add(TypeInfoUtils.getTypeInfoFromObjectInspector(arguments[i])); 
+    ArrayList<TypeInfo> argumentTypeInfos = new ArrayList<TypeInfo>(
+        arguments.length);
+    for (ObjectInspector argument : arguments) {
+      argumentTypeInfos.add(TypeInfoUtils
+          .getTypeInfoFromObjectInspector(argument));
     }
     udfMethod = udf.getResolver().getEvalMethod(argumentTypeInfos);
-    
+
     // Create parameter converters
     conversionHelper = new ConversionHelper(udfMethod, arguments);
 
     // Create the non-deferred realArgument
     realArguments = new Object[arguments.length];
-    
+
     // Get the return ObjectInspector.
-    ObjectInspector returnOI = ObjectInspectorFactory.getReflectionObjectInspector(
-        udfMethod.getGenericReturnType(), ObjectInspectorOptions.JAVA);
-    
+    ObjectInspector returnOI = ObjectInspectorFactory
+        .getReflectionObjectInspector(udfMethod.getGenericReturnType(),
+            ObjectInspectorOptions.JAVA);
+
     return returnOI;
   }
 
   @Override
   public Object evaluate(DeferredObject[] arguments) throws HiveException {
-    assert(arguments.length == realArguments.length);
-    
+    assert (arguments.length == realArguments.length);
+
     // Calculate all the arguments
     for (int i = 0; i < realArguments.length; i++) {
       realArguments[i] = arguments[i].get();
     }
 
     // Call the function
-    Object result = FunctionRegistry.invoke(udfMethod, udf,
-        conversionHelper.convertIfNecessary(realArguments));
-    
+    Object result = FunctionRegistry.invoke(udfMethod, udf, conversionHelper
+        .convertIfNecessary(realArguments));
+
     return result;
   }
 
@@ -180,7 +185,7 @@
       StringBuilder sb = new StringBuilder();
       sb.append(udfName);
       sb.append("(");
-      for(int i = 0; i < children.length; i++) {
+      for (int i = 0; i < children.length; i++) {
         sb.append(children[i]);
         if (i + 1 < children.length) {
           sb.append(", ");

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFCase.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFCase.java?rev=901644&r1=901643&r2=901644&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFCase.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFCase.java Thu Jan 21 10:37:58 2010
@@ -20,24 +20,18 @@
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hive.ql.exec.ExprNodeEvaluator;
 import org.apache.hadoop.hive.ql.exec.UDFArgumentTypeException;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
-import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils;
 import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector;
-import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils.ObjectInspectorCopyOption;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorUtils;
-import org.apache.hadoop.hive.serde2.objectinspector.primitive.VoidObjectInspector;
-import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
-import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils;
 
 /**
- * GenericUDF Class for SQL construct "CASE WHEN a THEN b WHEN c THEN d [ELSE f] END".
+ * GenericUDF Class for SQL construct
+ * "CASE WHEN a THEN b WHEN c THEN d [ELSE f] END".
  * 
- * NOTES:
- * 1. a and c should be boolean, or an exception will be thrown.
- * 2. b, d and f should have the same TypeInfo, or an exception will be thrown.
+ * NOTES: 1. a and c should be boolean, or an exception will be thrown. 2. b, d
+ * and f should have the same TypeInfo, or an exception will be thrown.
  */
 public class GenericUDFCase extends GenericUDF {
 
@@ -46,79 +40,80 @@
   ObjectInspector[] argumentOIs;
   GenericUDFUtils.ReturnObjectInspectorResolver returnOIResolver;
   GenericUDFUtils.ReturnObjectInspectorResolver caseOIResolver;
-  
+
   @Override
   public ObjectInspector initialize(ObjectInspector[] arguments)
       throws UDFArgumentTypeException {
-    
-    this.argumentOIs = arguments;
+
+    argumentOIs = arguments;
     caseOIResolver = new GenericUDFUtils.ReturnObjectInspectorResolver();
     returnOIResolver = new GenericUDFUtils.ReturnObjectInspectorResolver();
-    
+
     boolean r = caseOIResolver.update(arguments[0]);
-    assert(r);
-    for (int i=1; i+1<arguments.length; i+=2) {
+    assert (r);
+    for (int i = 1; i + 1 < arguments.length; i += 2) {
       if (!caseOIResolver.update(arguments[i])) {
         throw new UDFArgumentTypeException(i,
             "The expressions after WHEN should have the same type with that after CASE: \""
-            + caseOIResolver.get().getTypeName() + "\" is expected but \"" 
-            + arguments[i].getTypeName() + "\" is found");
+                + caseOIResolver.get().getTypeName() + "\" is expected but \""
+                + arguments[i].getTypeName() + "\" is found");
       }
-      if (!returnOIResolver.update(arguments[i+1])) {
-        throw new UDFArgumentTypeException(i+1,
+      if (!returnOIResolver.update(arguments[i + 1])) {
+        throw new UDFArgumentTypeException(i + 1,
             "The expressions after THEN should have the same type: \""
-            + returnOIResolver.get().getTypeName() + "\" is expected but \""
-            + arguments[i+1].getTypeName() + "\" is found");
+                + returnOIResolver.get().getTypeName()
+                + "\" is expected but \"" + arguments[i + 1].getTypeName()
+                + "\" is found");
       }
     }
     if (arguments.length % 2 == 0) {
       int i = arguments.length - 2;
-      if (!returnOIResolver.update(arguments[i+1])) {
-        throw new UDFArgumentTypeException(i+1,
+      if (!returnOIResolver.update(arguments[i + 1])) {
+        throw new UDFArgumentTypeException(i + 1,
             "The expression after ELSE should have the same type as those after THEN: \""
-            + returnOIResolver.get().getTypeName() + "\" is expected but \""
-            + arguments[i+1].getTypeName() + "\" is found");
+                + returnOIResolver.get().getTypeName()
+                + "\" is expected but \"" + arguments[i + 1].getTypeName()
+                + "\" is found");
       }
     }
-    
+
     return returnOIResolver.get();
   }
 
   @Override
   public Object evaluate(DeferredObject[] arguments) throws HiveException {
-    Object exprValue = arguments[0].get(); 
-    for (int i=1; i+1<arguments.length; i+=2) {
+    Object exprValue = arguments[0].get();
+    for (int i = 1; i + 1 < arguments.length; i += 2) {
       Object caseKey = arguments[i].get();
-      if (PrimitiveObjectInspectorUtils.comparePrimitiveObjects(
-          exprValue, (PrimitiveObjectInspector)argumentOIs[0],
-          caseKey, (PrimitiveObjectInspector)argumentOIs[i])) {
-        Object caseValue = arguments[i+1].get();
+      if (PrimitiveObjectInspectorUtils.comparePrimitiveObjects(exprValue,
+          (PrimitiveObjectInspector) argumentOIs[0], caseKey,
+          (PrimitiveObjectInspector) argumentOIs[i])) {
+        Object caseValue = arguments[i + 1].get();
         return returnOIResolver.convertIfNecessary(caseValue,
-            argumentOIs[i+1]);
+            argumentOIs[i + 1]);
       }
     }
     // Process else statement
     if (arguments.length % 2 == 0) {
       int i = arguments.length - 2;
-      Object elseValue = arguments[i+1].get();
-      return returnOIResolver.convertIfNecessary(elseValue,
-          argumentOIs[i+1]);
+      Object elseValue = arguments[i + 1].get();
+      return returnOIResolver.convertIfNecessary(elseValue, argumentOIs[i + 1]);
     }
     return null;
   }
 
   @Override
   public String getDisplayString(String[] children) {
-    assert(children.length >= 3);
+    assert (children.length >= 3);
     StringBuilder sb = new StringBuilder();
     sb.append("CASE (");
     sb.append(children[0]);
     sb.append(")");
-    for(int i=1; i+1<children.length; i+=2) {
+    for (int i = 1; i + 1 < children.length; i += 2) {
       sb.append(" WHEN (");
       sb.append(children[i]);
       sb.append(") THEN (");
-      sb.append(children[i+1]);
+      sb.append(children[i + 1]);
       sb.append(")");
     }
     if (children.length % 2 == 0) {

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFCoalesce.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFCoalesce.java?rev=901644&r1=901643&r2=901644&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFCoalesce.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFCoalesce.java Thu Jan 21 10:37:58 2010
@@ -20,52 +20,41 @@
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hive.ql.exec.ExprNodeEvaluator;
 import org.apache.hadoop.hive.ql.exec.UDFArgumentTypeException;
 import org.apache.hadoop.hive.ql.exec.description;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
-import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils;
-import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector;
-import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils.ObjectInspectorCopyOption;
-import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorUtils;
-import org.apache.hadoop.hive.serde2.objectinspector.primitive.VoidObjectInspector;
-import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
-import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils;
 
 /**
  * GenericUDF Class for SQL construct "COALESCE(a, b, c)".
  * 
- * NOTES:
- * 1. a, b and c should have the same TypeInfo, or an exception will be thrown.
+ * NOTES: 1. a, b and c should have the same TypeInfo, or an exception will be
+ * thrown.
  */
-@description(
-    name = "coalesce",
-    value = "_FUNC_(a1, a2, ...) - Returns the first non-null argument",
-    extended = "Example:\n" +
-        "  > SELECT _FUNC_(NULL, 1, NULL) FROM src LIMIT 1;\n" +
-        "  1"
-    )
+@description(name = "coalesce", value = "_FUNC_(a1, a2, ...) - Returns the first non-null argument", extended = "Example:\n"
+    + "  > SELECT _FUNC_(NULL, 1, NULL) FROM src LIMIT 1;\n" + "  1")
 public class GenericUDFCoalesce extends GenericUDF {
 
-  private static Log LOG = LogFactory.getLog(GenericUDFCoalesce.class.getName());
+  private static Log LOG = LogFactory
+      .getLog(GenericUDFCoalesce.class.getName());
 
   ObjectInspector[] argumentOIs;
   GenericUDFUtils.ReturnObjectInspectorResolver returnOIResolver;
-  
+
   @Override
   public ObjectInspector initialize(ObjectInspector[] arguments)
       throws UDFArgumentTypeException {
-    
-    this.argumentOIs = arguments;
-    
+
+    argumentOIs = arguments;
+
     returnOIResolver = new GenericUDFUtils.ReturnObjectInspectorResolver();
-    for (int i=0; i<arguments.length; i++) {
+    for (int i = 0; i < arguments.length; i++) {
       if (!returnOIResolver.update(arguments[i])) {
         throw new UDFArgumentTypeException(i,
             "The expressions after COALESCE should all have the same type: \""
-            + returnOIResolver.get().getTypeName() + "\" is expected but \"" 
-            + arguments[i].getTypeName() + "\" is found");
+                + returnOIResolver.get().getTypeName()
+                + "\" is expected but \"" + arguments[i].getTypeName()
+                + "\" is found");
       }
     }
     return returnOIResolver.get();
@@ -73,7 +62,7 @@
 
   @Override
   public Object evaluate(DeferredObject[] arguments) throws HiveException {
-    for (int i=0; i<arguments.length; i++) {
+    for (int i = 0; i < arguments.length; i++) {
       Object ai = arguments[i].get();
       if (ai == null) {
         continue;
@@ -89,7 +78,7 @@
     sb.append("COALESCE(");
     if (children.length > 0) {
       sb.append(children[0]);
-      for(int i=1; i<children.length; i++) {
+      for (int i = 1; i < children.length; i++) {
         sb.append(",");
         sb.append(children[i]);
       }

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFConcatWS.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFConcatWS.java?rev=901644&r1=901643&r2=901644&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFConcatWS.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFConcatWS.java Thu Jan 21 10:37:58 2010
@@ -30,53 +30,51 @@
 import org.apache.hadoop.io.Text;
 
 /**
- * Generic UDF for string function <code>CONCAT_WS(sep,str1,str2,str3,...)</code>.
- * This mimics the function from MySQL
- * http://dev.mysql.com/doc/refman/5.0/en/string-functions.html#function_concat-ws
- *
+ * Generic UDF for string function
+ * <code>CONCAT_WS(sep,str1,str2,str3,...)</code>. This mimics the function from
+ * MySQL http://dev.mysql.com/doc/refman/5.0/en/string-functions.html#
+ * function_concat-ws
+ * 
  * @see org.apache.hadoop.hive.ql.udf.generic.GenericUDF
  */
-@description(
-    name = "concat_ws",
-    value = "_FUNC_(separator, str1, str2, ...) - " +
-            "returns the concatenation of the strings separated by the separator.",
-    extended = "Example:\n" +
-      "  > SELECT _FUNC_('ce', 'fa', 'book') FROM src LIMIT 1;\n" +
-      "  'facebook'")
-
+@description(name = "concat_ws", value = "_FUNC_(separator, str1, str2, ...) - "
+    + "returns the concatenation of the strings separated by the separator.", extended = "Example:\n"
+    + "  > SELECT _FUNC_('ce', 'fa', 'book') FROM src LIMIT 1;\n"
+    + "  'facebook'")
 public class GenericUDFConcatWS extends GenericUDF {
 
   ObjectInspector[] argumentOIs;
 
   @Override
   public ObjectInspector initialize(ObjectInspector[] arguments)
-    throws UDFArgumentException {
+      throws UDFArgumentException {
     if (arguments.length < 2) {
       throw new UDFArgumentLengthException(
-        "The function CONCAT_WS(separator,str1,str2,str3,...) needs at least two arguments.");
+          "The function CONCAT_WS(separator,str1,str2,str3,...) needs at least two arguments.");
     }
 
     for (int i = 0; i < arguments.length; i++) {
-      if(arguments[i].getTypeName() != Constants.STRING_TYPE_NAME
-         && arguments[i].getTypeName() != Constants.VOID_TYPE_NAME) {
-        throw new UDFArgumentTypeException(i,
-                                           "Argument " + (i +1 )  + " of function CONCAT_WS must be \"" + Constants.STRING_TYPE_NAME
-                                           + "\", but \"" + arguments[i].getTypeName() + "\" was found.");
+      if (arguments[i].getTypeName() != Constants.STRING_TYPE_NAME
+          && arguments[i].getTypeName() != Constants.VOID_TYPE_NAME) {
+        throw new UDFArgumentTypeException(i, "Argument " + (i + 1)
+            + " of function CONCAT_WS must be \"" + Constants.STRING_TYPE_NAME
+            + "\", but \"" + arguments[i].getTypeName() + "\" was found.");
       }
     }
 
-    this.argumentOIs = arguments;
+    argumentOIs = arguments;
     return PrimitiveObjectInspectorFactory.writableStringObjectInspector;
   }
 
-  private Text resultText = new Text();
+  private final Text resultText = new Text();
+
   @Override
   public Object evaluate(DeferredObject[] arguments) throws HiveException {
     if (arguments[0].get() == null) {
       return null;
     }
-    String separator =
-      ((StringObjectInspector)argumentOIs[0]).getPrimitiveJavaObject(arguments[0].get());
+    String separator = ((StringObjectInspector) argumentOIs[0])
+        .getPrimitiveJavaObject(arguments[0].get());
 
     StringBuilder sb = new StringBuilder();
     boolean first = true;
@@ -87,7 +85,8 @@
         } else {
           sb.append(separator);
         }
-        sb.append(((StringObjectInspector)argumentOIs[i]).getPrimitiveJavaObject(arguments[i].get()));
+        sb.append(((StringObjectInspector) argumentOIs[i])
+            .getPrimitiveJavaObject(arguments[i].get()));
       }
     }
 

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFElt.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFElt.java?rev=901644&r1=901643&r2=901644&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFElt.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFElt.java Thu Jan 21 10:37:58 2010
@@ -28,22 +28,16 @@
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector.Category;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
 import org.apache.hadoop.io.IntWritable;
-import org.apache.hadoop.io.Text;
 
 /**
- * Generic UDF for string function <code>ELT(N,str1,str2,str3,...)</code>.
- * This mimics the function from MySQL
+ * Generic UDF for string function <code>ELT(N,str1,str2,str3,...)</code>. This
+ * mimics the function from MySQL
  * http://dev.mysql.com/doc/refman/5.1/en/string-functions.html#function_elt
  * 
  * @see org.apache.hadoop.hive.ql.udf.generic.GenericUDF
  */
-@description(
-    name = "elt",
-    value = "_FUNC_(n, str1, str2, ...) - returns the n-th string",
-    extended = "Example:\n" +
-        "  > SELECT _FUNC_(1, 'face', 'book') FROM src LIMIT 1;\n" +
-        "  'face'"
-    )
+@description(name = "elt", value = "_FUNC_(n, str1, str2, ...) - returns the n-th string", extended = "Example:\n"
+    + "  > SELECT _FUNC_(1, 'face', 'book') FROM src LIMIT 1;\n" + "  'face'")
 public class GenericUDFElt extends GenericUDF {
 
   ObjectInspectorConverters.Converter[] converters;
@@ -56,19 +50,20 @@
           "The function ELT(N,str1,str2,str3,...) needs at least two arguments.");
     }
 
-    for(int i = 0; i < arguments.length; i++) {
+    for (int i = 0; i < arguments.length; i++) {
       Category category = arguments[i].getCategory();
-      if(category != Category.PRIMITIVE) {
-        throw new UDFArgumentTypeException(i,
-            "The " + GenericUDFUtils.getOrdinal(i + 1) + " argument of function ELT is expected to a " 
-            + Category.PRIMITIVE.toString().toLowerCase()
-            + " type, but " + category.toString().toLowerCase() + " is found");
+      if (category != Category.PRIMITIVE) {
+        throw new UDFArgumentTypeException(i, "The "
+            + GenericUDFUtils.getOrdinal(i + 1)
+            + " argument of function ELT is expected to a "
+            + Category.PRIMITIVE.toString().toLowerCase() + " type, but "
+            + category.toString().toLowerCase() + " is found");
       }
     }
 
     converters = new ObjectInspectorConverters.Converter[arguments.length];
-    for(int i = 0; i < arguments.length; i++) {
-      if(i == 0) {
+    for (int i = 0; i < arguments.length; i++) {
+      if (i == 0) {
         converters[i] = ObjectInspectorConverters.getConverter(arguments[i],
             PrimitiveObjectInspectorFactory.writableIntObjectInspector);
       } else {
@@ -82,14 +77,16 @@
 
   @Override
   public Object evaluate(DeferredObject[] arguments) throws HiveException {
-    IntWritable intWritable = (IntWritable)converters[0].convert(arguments[0].get());
-    if(intWritable == null) {
+    IntWritable intWritable = (IntWritable) converters[0].convert(arguments[0]
+        .get());
+    if (intWritable == null) {
       return null;
     }
     int index = intWritable.get();
-    if (index <= 0 || index >= arguments.length)
+    if (index <= 0 || index >= arguments.length) {
       return null;
-    return (Text) converters[index].convert(arguments[index].get());
+    }
+    return converters[index].convert(arguments[index].get());
   }
 
   @Override

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFField.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFField.java?rev=901644&r1=901643&r2=901644&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFField.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFField.java Thu Jan 21 10:37:58 2010
@@ -28,32 +28,32 @@
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
 import org.apache.hadoop.io.IntWritable;
 
-@description(
-    name="field",
-    value = "_FUNC_(str, str1, str2, ...) - returns the index of str in the str1,str2,... list or 0 if not found",
-    extended = "All primitive types are supported, arguments are compared using str.equals(x)." +
-    " If str is NULL, the return value is 0."
-    )
+@description(name = "field", value = "_FUNC_(str, str1, str2, ...) - returns the index of str in the str1,str2,... list or 0 if not found", extended = "All primitive types are supported, arguments are compared using str.equals(x)."
+    + " If str is NULL, the return value is 0.")
 public class GenericUDFField extends GenericUDF {
-  public ObjectInspector initialize(ObjectInspector[] arguments) throws UDFArgumentException {
+  @Override
+  public ObjectInspector initialize(ObjectInspector[] arguments)
+      throws UDFArgumentException {
     if (arguments.length < 2) {
-      throw new UDFArgumentException("The function FIELD(str, str1, str2, ...) needs at least two arguments.");
+      throw new UDFArgumentException(
+          "The function FIELD(str, str1, str2, ...) needs at least two arguments.");
     }
-    
+
     for (int i = 0; i < arguments.length; i++) {
       Category category = arguments[i].getCategory();
       if (category != Category.PRIMITIVE) {
-        throw new UDFArgumentTypeException(i,
-            "The " + GenericUDFUtils.getOrdinal(i + 1) + " argument of function FIELD is expected to a " 
-            + Category.PRIMITIVE.toString().toLowerCase()
-            + " type, but " + category.toString().toLowerCase() + " is found");
+        throw new UDFArgumentTypeException(i, "The "
+            + GenericUDFUtils.getOrdinal(i + 1)
+            + " argument of function FIELD is expected to a "
+            + Category.PRIMITIVE.toString().toLowerCase() + " type, but "
+            + category.toString().toLowerCase() + " is found");
       }
     }
-    
+
     return PrimitiveObjectInspectorFactory.writableIntObjectInspector;
   }
-  
-  private IntWritable r = new IntWritable();
+
+  private final IntWritable r = new IntWritable();
 
   @Override
   public Object evaluate(DeferredObject[] arguments) throws HiveException {
@@ -61,27 +61,27 @@
       r.set(0);
       return r;
     }
-    
-    for (int i=1; i< arguments.length; i++) {
+
+    for (int i = 1; i < arguments.length; i++) {
       if (arguments[0].get().equals(arguments[i].get())) {
         r.set(i);
         return r;
       }
     }
-    
+
     r.set(0);
     return r;
   }
 
   @Override
   public String getDisplayString(String[] children) {
-    assert(children.length >= 2);
-    
+    assert (children.length >= 2);
+
     final StringBuilder sb = new StringBuilder();
     sb.append("field(");
     sb.append(StringUtils.join(children, ", "));
     sb.append(")");
-    
+
     return sb.toString();
   }
 }

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFHash.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFHash.java?rev=901644&r1=901643&r2=901644&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFHash.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFHash.java Thu Jan 21 10:37:58 2010
@@ -21,49 +21,42 @@
 import org.apache.commons.lang.StringUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hive.ql.exec.ExprNodeEvaluator;
 import org.apache.hadoop.hive.ql.exec.UDFArgumentTypeException;
 import org.apache.hadoop.hive.ql.exec.description;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils;
-import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector;
-import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils.ObjectInspectorCopyOption;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
-import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorUtils;
-import org.apache.hadoop.hive.serde2.objectinspector.primitive.VoidObjectInspector;
-import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
-import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils;
 import org.apache.hadoop.io.IntWritable;
 
 /**
  * GenericUDF Class for computing hash values.
  */
-@description(
-    name = "hash",
-    value = "_FUNC_(a1, a2, ...) - Returns a hash value of the arguments"
-)
+@description(name = "hash", value = "_FUNC_(a1, a2, ...) - Returns a hash value of the arguments")
 public class GenericUDFHash extends GenericUDF {
 
   private static Log LOG = LogFactory.getLog(GenericUDFHash.class.getName());
 
   ObjectInspector[] argumentOIs;
-  
+
   @Override
   public ObjectInspector initialize(ObjectInspector[] arguments)
       throws UDFArgumentTypeException {
-    
-    this.argumentOIs = arguments;
+
+    argumentOIs = arguments;
     return PrimitiveObjectInspectorFactory.writableIntObjectInspector;
   }
 
   IntWritable result = new IntWritable();
+
   @Override
   public Object evaluate(DeferredObject[] arguments) throws HiveException {
-    // See http://java.sun.com/j2se/1.5.0/docs/api/java/util/List.html#hashCode()
+    // See
+    // http://java.sun.com/j2se/1.5.0/docs/api/java/util/List.html#hashCode()
     int r = 0;
-    for(int i = 0; i < arguments.length; i++) {
-      r = r * 31 + ObjectInspectorUtils.hashCode(arguments[i].get(), argumentOIs[i]);
+    for (int i = 0; i < arguments.length; i++) {
+      r = r * 31
+          + ObjectInspectorUtils.hashCode(arguments[i].get(), argumentOIs[i]);
     }
     result.set(r);
     return result;

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFIf.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFIf.java?rev=901644&r1=901643&r2=901644&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFIf.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFIf.java Thu Jan 21 10:37:58 2010
@@ -24,13 +24,14 @@
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.serde.Constants;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
-import org.apache.hadoop.hive.serde2.objectinspector.primitive.BooleanObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.BooleanObjectInspector;
 
 /**
  * IF(expr1,expr2,expr3) <br>
- * If expr1 is TRUE (expr1 <> 0 and expr1 <> NULL) then IF() returns expr2; otherwise it returns expr3. 
- * IF() returns a numeric or string value, depending on the context in which it is used. 
+ * If expr1 is TRUE (expr1 <> 0 and expr1 <> NULL) then IF() returns expr2;
+ * otherwise it returns expr3. IF() returns a numeric or string value, depending
+ * on the context in which it is used.
  */
 public class GenericUDFIf extends GenericUDF {
 
@@ -40,7 +41,7 @@
   @Override
   public ObjectInspector initialize(ObjectInspector[] arguments)
       throws UDFArgumentException {
-    this.argumentOIs = arguments;
+    argumentOIs = arguments;
     returnOIResolver = new GenericUDFUtils.ReturnObjectInspectorResolver(true);
 
     if (arguments.length != 3) {
@@ -50,22 +51,23 @@
 
     boolean conditionTypeIsOk = (arguments[0].getCategory() == ObjectInspector.Category.PRIMITIVE);
     if (conditionTypeIsOk) {
-      PrimitiveObjectInspector poi = ((PrimitiveObjectInspector)arguments[0]);
-      conditionTypeIsOk = (poi.getPrimitiveCategory() == PrimitiveObjectInspector.PrimitiveCategory.BOOLEAN
-                           || poi.getPrimitiveCategory() == PrimitiveObjectInspector.PrimitiveCategory.VOID);
+      PrimitiveObjectInspector poi = ((PrimitiveObjectInspector) arguments[0]);
+      conditionTypeIsOk = (poi.getPrimitiveCategory() == PrimitiveObjectInspector.PrimitiveCategory.BOOLEAN || poi
+          .getPrimitiveCategory() == PrimitiveObjectInspector.PrimitiveCategory.VOID);
     }
     if (!conditionTypeIsOk) {
       throw new UDFArgumentTypeException(0,
-          "The first argument of function IF should be \"" + Constants.BOOLEAN_TYPE_NAME
-          + "\", but \"" + arguments[0].getTypeName() + "\" is found");
+          "The first argument of function IF should be \""
+              + Constants.BOOLEAN_TYPE_NAME + "\", but \""
+              + arguments[0].getTypeName() + "\" is found");
     }
 
-    if( !(returnOIResolver.update(arguments[1]) 
-         && returnOIResolver.update(arguments[2])) ) {
+    if (!(returnOIResolver.update(arguments[1]) && returnOIResolver
+        .update(arguments[2]))) {
       throw new UDFArgumentTypeException(2,
-          "The second and the third arguments of function IF should have the same type, " +
-          "but they are different: \"" + arguments[1].getTypeName() 
-          + "\" and \"" + arguments[2].getTypeName() + "\"");
+          "The second and the third arguments of function IF should have the same type, "
+              + "but they are different: \"" + arguments[1].getTypeName()
+              + "\" and \"" + arguments[2].getTypeName() + "\"");
     }
 
     return returnOIResolver.get();
@@ -74,7 +76,8 @@
   @Override
   public Object evaluate(DeferredObject[] arguments) throws HiveException {
     Object condition = arguments[0].get();
-    if(condition != null && ((BooleanObjectInspector)argumentOIs[0]).get(condition)) {
+    if (condition != null
+        && ((BooleanObjectInspector) argumentOIs[0]).get(condition)) {
       return returnOIResolver.convertIfNecessary(arguments[1].get(),
           argumentOIs[1]);
     } else {

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFIndex.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFIndex.java?rev=901644&r1=901643&r2=901644&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFIndex.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFIndex.java Thu Jan 21 10:37:58 2010
@@ -23,27 +23,22 @@
 import org.apache.hadoop.hive.ql.exec.UDFArgumentTypeException;
 import org.apache.hadoop.hive.ql.exec.description;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
-import org.apache.hadoop.hive.serde.Constants;
 import org.apache.hadoop.hive.serde2.objectinspector.ListObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.MapObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector.Category;
-import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorUtils;
 import org.apache.hadoop.io.IntWritable;
 
-@description(
-    name = "index",
-    value = "_FUNC_(a, n) - Returns the n-th element of a "
-)
+@description(name = "index", value = "_FUNC_(a, n) - Returns the n-th element of a ")
 public class GenericUDFIndex extends GenericUDF {
   private MapObjectInspector mapOI;
   private boolean mapKeyPreferWritable;
   private ListObjectInspector listOI;
   private PrimitiveObjectInspector indexOI;
   private ObjectInspector returnOI;
-  private IntWritable result = new IntWritable(-1);
+  private final IntWritable result = new IntWritable(-1);
 
   @Override
   public ObjectInspector initialize(ObjectInspector[] arguments)
@@ -52,51 +47,51 @@
       throw new UDFArgumentLengthException(
           "The function INDEX accepts exactly 2 arguments.");
     }
-    
+
     if (arguments[0] instanceof MapObjectInspector) {
       // index into a map
-      mapOI = (MapObjectInspector)arguments[0];
+      mapOI = (MapObjectInspector) arguments[0];
       listOI = null;
     } else if (arguments[0] instanceof ListObjectInspector) {
       // index into a list
-      listOI = (ListObjectInspector)arguments[0];
+      listOI = (ListObjectInspector) arguments[0];
       mapOI = null;
     } else {
-      throw new UDFArgumentTypeException(0,
-          "\"" + Category.MAP.toString().toLowerCase() 
-          + "\" or \"" + Category.LIST.toString().toLowerCase() 
-          + "\" is expected at function INDEX, but \"" 
+      throw new UDFArgumentTypeException(0, "\""
+          + Category.MAP.toString().toLowerCase() + "\" or \""
+          + Category.LIST.toString().toLowerCase()
+          + "\" is expected at function INDEX, but \""
           + arguments[0].getTypeName() + "\" is found");
     }
-    
+
     // index has to be a primitive
     if (arguments[1] instanceof PrimitiveObjectInspector) {
       indexOI = (PrimitiveObjectInspector) arguments[1];
     } else {
-      throw new UDFArgumentTypeException(1,
-          "Primitive Type is expected but " + arguments[1].getTypeName()
-          + "\" is found");
+      throw new UDFArgumentTypeException(1, "Primitive Type is expected but "
+          + arguments[1].getTypeName() + "\" is found");
     }
 
     if (mapOI != null) {
       returnOI = mapOI.getMapValueObjectInspector();
       ObjectInspector keyOI = mapOI.getMapKeyObjectInspector();
-      mapKeyPreferWritable = ((PrimitiveObjectInspector)keyOI).preferWritable();
+      mapKeyPreferWritable = ((PrimitiveObjectInspector) keyOI)
+          .preferWritable();
     } else {
       returnOI = listOI.getListElementObjectInspector();
     }
-    
+
     return returnOI;
   }
 
   @Override
   public Object evaluate(DeferredObject[] arguments) throws HiveException {
-    assert(arguments.length == 2);
+    assert (arguments.length == 2);
     Object main = arguments[0].get();
     Object index = arguments[1].get();
 
     if (mapOI != null) {
-      
+
       Object indexObject;
       if (mapKeyPreferWritable) {
         indexObject = indexOI.getPrimitiveWritableObject(index);
@@ -104,10 +99,10 @@
         indexObject = indexOI.getPrimitiveJavaObject(index);
       }
       return mapOI.getMapValueElement(main, indexObject);
-      
+
     } else {
-      
-      assert(listOI != null);
+
+      assert (listOI != null);
       int intIndex = 0;
       try {
         intIndex = PrimitiveObjectInspectorUtils.getInt(index, indexOI);
@@ -119,13 +114,13 @@
         return null;
       }
       return listOI.getListElement(main, intIndex);
-      
+
     }
   }
 
   @Override
   public String getDisplayString(String[] children) {
-    assert(children.length == 2);
+    assert (children.length == 2);
     return children[0] + "[" + children[1] + "]";
   }
 }

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFInstr.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFInstr.java?rev=901644&r1=901643&r2=901644&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFInstr.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFInstr.java Thu Jan 21 10:37:58 2010
@@ -31,26 +31,23 @@
 import org.apache.hadoop.io.Text;
 
 /**
- * Generic UDF for string function <code>INSTR(str,substr)</code>.
- * This mimcs the function from MySQL
+ * Generic UDF for string function <code>INSTR(str,substr)</code>. This mimcs
+ * the function from MySQL
  * http://dev.mysql.com/doc/refman/5.1/en/string-functions.html#function_instr
- * <pre> 
+ * 
+ * <pre>
  * usage:
  * INSTR(str, substr)
- * </pre><p>
+ * </pre>
+ * <p>
  */
-@description(
-    name = "instr",
-    value = "_FUNC_(str, substr) - Returns the index of the first occurance " +
-    		"of substr in str",
-    extended = "Example:\n" +
-        "  > SELECT _FUNC_('Facebook', 'boo') FROM src LIMIT 1;\n" +
-        "  5"
-    )
-public class GenericUDFInstr extends GenericUDF{
+@description(name = "instr", value = "_FUNC_(str, substr) - Returns the index of the first occurance "
+    + "of substr in str", extended = "Example:\n"
+    + "  > SELECT _FUNC_('Facebook', 'boo') FROM src LIMIT 1;\n" + "  5")
+public class GenericUDFInstr extends GenericUDF {
 
   ObjectInspectorConverters.Converter[] converters;
-  
+
   @Override
   public ObjectInspector initialize(ObjectInspector[] arguments)
       throws UDFArgumentException {
@@ -59,40 +56,43 @@
           "The function INSTR accepts exactly 2 arguments.");
     }
 
-    for(int i = 0; i < arguments.length; i++) {
+    for (int i = 0; i < arguments.length; i++) {
       Category category = arguments[i].getCategory();
-      if(category != Category.PRIMITIVE) {
-        throw new UDFArgumentTypeException(i,
-            "The " + GenericUDFUtils.getOrdinal(i + 1) + " argument of function INSTR is expected to a " 
-            + Category.PRIMITIVE.toString().toLowerCase()
-            + " type, but " + category.toString().toLowerCase() + " is found");
+      if (category != Category.PRIMITIVE) {
+        throw new UDFArgumentTypeException(i, "The "
+            + GenericUDFUtils.getOrdinal(i + 1)
+            + " argument of function INSTR is expected to a "
+            + Category.PRIMITIVE.toString().toLowerCase() + " type, but "
+            + category.toString().toLowerCase() + " is found");
       }
     }
 
     converters = new ObjectInspectorConverters.Converter[arguments.length];
-    for(int i = 0; i < arguments.length; i++) {
-        converters[i] = ObjectInspectorConverters.getConverter(arguments[i],
-            PrimitiveObjectInspectorFactory.writableStringObjectInspector);
+    for (int i = 0; i < arguments.length; i++) {
+      converters[i] = ObjectInspectorConverters.getConverter(arguments[i],
+          PrimitiveObjectInspectorFactory.writableStringObjectInspector);
     }
 
     return PrimitiveObjectInspectorFactory.writableIntObjectInspector;
   }
-  
+
   IntWritable intWritable = new IntWritable(0);
+
   @Override
   public Object evaluate(DeferredObject[] arguments) throws HiveException {
-    if(arguments[0].get() == null || arguments[1].get() == null)
+    if (arguments[0].get() == null || arguments[1].get() == null) {
       return null;
+    }
 
     Text text = (Text) converters[0].convert(arguments[0].get());
     Text subtext = (Text) converters[1].convert(arguments[1].get());
     intWritable.set(GenericUDFUtils.findText(text, subtext, 0) + 1);
-    return  intWritable;
+    return intWritable;
   }
 
   @Override
   public String getDisplayString(String[] children) {
-    assert(children.length == 2);
+    assert (children.length == 2);
     return "instr(" + children[0] + children[1] + ")";
   }
 }

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFLocate.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFLocate.java?rev=901644&r1=901643&r2=901644&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFLocate.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFLocate.java Thu Jan 21 10:37:58 2010
@@ -31,27 +31,24 @@
 import org.apache.hadoop.io.Text;
 
 /**
- * Generic UDF for string function <code>LOCATE(substr, str)</code>, <code>LOCATE(substr, str, start)</code>.
- * This mimcs the function from MySQL
+ * Generic UDF for string function <code>LOCATE(substr, str)</code>,
+ * <code>LOCATE(substr, str, start)</code>. This mimcs the function from MySQL
  * http://dev.mysql.com/doc/refman/5.1/en/string-functions.html#function_locate
- * <pre> 
+ * 
+ * <pre>
  * usage:
  * LOCATE(substr, str)
  * LOCATE(substr, str, start)
- * </pre><p>
+ * </pre>
+ * <p>
  */
-@description(
-    name = "locate",
-    value = "_FUNC_(substr, str[, pos]) - Returns the position of the first " +
-    		"occurance of substr in str after position pos",
-    extended = "Example:\n" +
-        "  > SELECT _FUNC_('bar', 'foobarbar', 5) FROM src LIMIT 1;\n" +
-        "  7"
-    )
-public class GenericUDFLocate extends GenericUDF{
+@description(name = "locate", value = "_FUNC_(substr, str[, pos]) - Returns the position of the first "
+    + "occurance of substr in str after position pos", extended = "Example:\n"
+    + "  > SELECT _FUNC_('bar', 'foobarbar', 5) FROM src LIMIT 1;\n" + "  7")
+public class GenericUDFLocate extends GenericUDF {
 
   ObjectInspectorConverters.Converter[] converters;
-  
+
   @Override
   public ObjectInspector initialize(ObjectInspector[] arguments)
       throws UDFArgumentException {
@@ -60,22 +57,23 @@
           "The function LOCATE accepts exactly 2 or 3 arguments.");
     }
 
-    for(int i = 0; i < arguments.length; i++) {
+    for (int i = 0; i < arguments.length; i++) {
       Category category = arguments[i].getCategory();
-      if(category != Category.PRIMITIVE) {
-        throw new UDFArgumentTypeException(i,
-            "The " + GenericUDFUtils.getOrdinal(i + 1) + " argument of function LOCATE is expected to a " 
-            + Category.PRIMITIVE.toString().toLowerCase()
-            + " type, but " + category.toString().toLowerCase() + " is found");
+      if (category != Category.PRIMITIVE) {
+        throw new UDFArgumentTypeException(i, "The "
+            + GenericUDFUtils.getOrdinal(i + 1)
+            + " argument of function LOCATE is expected to a "
+            + Category.PRIMITIVE.toString().toLowerCase() + " type, but "
+            + category.toString().toLowerCase() + " is found");
       }
     }
 
     converters = new ObjectInspectorConverters.Converter[arguments.length];
-    for(int i = 0; i < arguments.length; i++) {
-      if(i == 0 || i == 1) {
+    for (int i = 0; i < arguments.length; i++) {
+      if (i == 0 || i == 1) {
         converters[i] = ObjectInspectorConverters.getConverter(arguments[i],
             PrimitiveObjectInspectorFactory.writableStringObjectInspector);
-      } else if(i == 2) {
+      } else if (i == 2) {
         converters[i] = ObjectInspectorConverters.getConverter(arguments[i],
             PrimitiveObjectInspectorFactory.writableIntObjectInspector);
       }
@@ -83,32 +81,35 @@
 
     return PrimitiveObjectInspectorFactory.writableIntObjectInspector;
   }
-  
+
   IntWritable intWritable = new IntWritable(0);
+
   @Override
   public Object evaluate(DeferredObject[] arguments) throws HiveException {
-    if(arguments[0].get() == null || arguments[1].get() == null)
+    if (arguments[0].get() == null || arguments[1].get() == null) {
       return null;
+    }
 
     Text subtext = (Text) converters[0].convert(arguments[0].get());
     Text text = (Text) converters[1].convert(arguments[1].get());
     int start = 1;
-    if(arguments.length == 3) {
-        IntWritable startWritable = (IntWritable)converters[2].convert(arguments[2].get());
-        if(startWritable == null) {
-          intWritable.set(0);
-          return intWritable;
-        }
-        start = startWritable.get();
+    if (arguments.length == 3) {
+      IntWritable startWritable = (IntWritable) converters[2]
+          .convert(arguments[2].get());
+      if (startWritable == null) {
+        intWritable.set(0);
+        return intWritable;
+      }
+      start = startWritable.get();
     }
     intWritable.set(GenericUDFUtils.findText(text, subtext, start - 1) + 1);
-    return  intWritable;
+    return intWritable;
   }
 
   @Override
   public String getDisplayString(String[] children) {
-    assert(children.length == 2 || children.length == 3);
-    return "locate(" + children[0] + children[1] 
-           + (children.length == 3 ? children[2] : "") + ")";
+    assert (children.length == 2 || children.length == 3);
+    return "locate(" + children[0] + children[1]
+        + (children.length == 3 ? children[2] : "") + ")";
   }
 }

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFMap.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFMap.java?rev=901644&r1=901643&r2=901644&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFMap.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFMap.java Thu Jan 21 10:37:58 2010
@@ -31,81 +31,81 @@
 import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorConverters.Converter;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
-import org.apache.hadoop.io.Text;
-
-@description(
-    name = "map",
-    value = "_FUNC_(key0, value0, key1, value1...) - Creates a map with the given key/value pairs "
-)
 
+@description(name = "map", value = "_FUNC_(key0, value0, key1, value1...) - Creates a map with the given key/value pairs ")
 public class GenericUDFMap extends GenericUDF {
   Converter[] converters;
   HashMap<Object, Object> ret = new HashMap<Object, Object>();
-  
+
   @Override
   public ObjectInspector initialize(ObjectInspector[] arguments)
-  throws UDFArgumentException {
-    
+      throws UDFArgumentException {
+
     if (arguments.length % 2 != 0) {
       throw new UDFArgumentLengthException(
           "Arguments must be in key/value pairs");
     }
 
-    GenericUDFUtils.ReturnObjectInspectorResolver keyOIResolver = 
-      new GenericUDFUtils.ReturnObjectInspectorResolver(true);
-    GenericUDFUtils.ReturnObjectInspectorResolver valueOIResolver = 
-      new GenericUDFUtils.ReturnObjectInspectorResolver(true);
+    GenericUDFUtils.ReturnObjectInspectorResolver keyOIResolver = new GenericUDFUtils.ReturnObjectInspectorResolver(
+        true);
+    GenericUDFUtils.ReturnObjectInspectorResolver valueOIResolver = new GenericUDFUtils.ReturnObjectInspectorResolver(
+        true);
 
-    for(int i=0; i<arguments.length; i++) {
-      if(i%2==0) {
+    for (int i = 0; i < arguments.length; i++) {
+      if (i % 2 == 0) {
         // Keys
         if (!(arguments[i] instanceof PrimitiveObjectInspector)) {
           throw new UDFArgumentTypeException(1,
               "Primitive Type is expected but " + arguments[i].getTypeName()
-              + "\" is found");
+                  + "\" is found");
         }
-        if(!keyOIResolver.update(arguments[i])) {
-          throw new UDFArgumentTypeException(i, "Key type \"" + arguments[i].getTypeName() + 
-              "\" is different from preceding key types. " + 
-              "Previous key type was \"" + arguments[i-2].getTypeName() + "\"");
+        if (!keyOIResolver.update(arguments[i])) {
+          throw new UDFArgumentTypeException(i, "Key type \""
+              + arguments[i].getTypeName()
+              + "\" is different from preceding key types. "
+              + "Previous key type was \"" + arguments[i - 2].getTypeName()
+              + "\"");
         }
       } else {
         // Values
-        if(!valueOIResolver.update(arguments[i])) {
-          throw new UDFArgumentTypeException(i, "Value type \"" + arguments[i].getTypeName() + 
-              "\" is different from preceding value types. " + 
-              "Previous value type was \"" + arguments[i-2].getTypeName() + "\"");
+        if (!valueOIResolver.update(arguments[i])) {
+          throw new UDFArgumentTypeException(i, "Value type \""
+              + arguments[i].getTypeName()
+              + "\" is different from preceding value types. "
+              + "Previous value type was \"" + arguments[i - 2].getTypeName()
+              + "\"");
         }
       }
     }
-    
+
     ObjectInspector keyOI = keyOIResolver.get();
     ObjectInspector valueOI = valueOIResolver.get();
-    
-    if(keyOI == null) {
-      keyOI = PrimitiveObjectInspectorFactory.getPrimitiveJavaObjectInspector(PrimitiveObjectInspector.PrimitiveCategory.STRING);
-    }
-    if(valueOI == null) {
-      valueOI = PrimitiveObjectInspectorFactory.getPrimitiveJavaObjectInspector(PrimitiveObjectInspector.PrimitiveCategory.STRING);
+
+    if (keyOI == null) {
+      keyOI = PrimitiveObjectInspectorFactory
+          .getPrimitiveJavaObjectInspector(PrimitiveObjectInspector.PrimitiveCategory.STRING);
+    }
+    if (valueOI == null) {
+      valueOI = PrimitiveObjectInspectorFactory
+          .getPrimitiveJavaObjectInspector(PrimitiveObjectInspector.PrimitiveCategory.STRING);
     }
-    
+
     converters = new Converter[arguments.length];
-    
-    for(int i=0; i<arguments.length; i++) {
-      converters[i] = ObjectInspectorConverters.getConverter(arguments[i], 
-        i%2==0 ? keyOI : valueOI);
-    }
-    
-    return ObjectInspectorFactory.getStandardMapObjectInspector(
-        keyOI, valueOI);
+
+    for (int i = 0; i < arguments.length; i++) {
+      converters[i] = ObjectInspectorConverters.getConverter(arguments[i],
+          i % 2 == 0 ? keyOI : valueOI);
+    }
+
+    return ObjectInspectorFactory.getStandardMapObjectInspector(keyOI, valueOI);
   }
 
   @Override
   public Object evaluate(DeferredObject[] arguments) throws HiveException {
-    ret.clear();    
-    for(int i=0; i<arguments.length; i+=2) {
-      ret.put(converters[i].convert(arguments[i].get()),
-          converters[i+1].convert(arguments[i+1].get()));
+    ret.clear();
+    for (int i = 0; i < arguments.length; i += 2) {
+      ret.put(converters[i].convert(arguments[i].get()), converters[i + 1]
+          .convert(arguments[i + 1].get()));
     }
     return ret;
   }
@@ -114,12 +114,12 @@
   public String getDisplayString(String[] children) {
     StringBuilder sb = new StringBuilder();
     sb.append("map(");
-    assert(children.length%2 == 0);
-    for(int i=0; i<children.length; i+=2) {
+    assert (children.length % 2 == 0);
+    for (int i = 0; i < children.length; i += 2) {
       sb.append(children[i]);
       sb.append(":");
-      sb.append(children[i+1]);
-      if(i+2 != children.length) {
+      sb.append(children[i + 1]);
+      if (i + 2 != children.length) {
         sb.append(",");
       }
     }

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFOPNotNull.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFOPNotNull.java?rev=901644&r1=901643&r2=901644&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFOPNotNull.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFOPNotNull.java Thu Jan 21 10:37:58 2010
@@ -26,10 +26,7 @@
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
 import org.apache.hadoop.io.BooleanWritable;
 
-@description(
-    name = "isnotnull",
-    value = "_FUNC_ a - Returns true if a is not NULL and false otherwise"
-)
+@description(name = "isnotnull", value = "_FUNC_ a - Returns true if a is not NULL and false otherwise")
 public class GenericUDFOPNotNull extends GenericUDF {
 
   BooleanWritable result = new BooleanWritable();

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFOPNull.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFOPNull.java?rev=901644&r1=901643&r2=901644&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFOPNull.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFOPNull.java Thu Jan 21 10:37:58 2010
@@ -26,10 +26,7 @@
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
 import org.apache.hadoop.io.BooleanWritable;
 
-@description(
-    name = "isnull",
-    value = "_FUNC_ a - Returns true if a is NULL and false otherwise"
-)
+@description(name = "isnull", value = "_FUNC_ a - Returns true if a is NULL and false otherwise")
 public class GenericUDFOPNull extends GenericUDF {
 
   BooleanWritable result = new BooleanWritable();

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFSize.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFSize.java?rev=901644&r1=901643&r2=901644&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFSize.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFSize.java Thu Jan 21 10:37:58 2010
@@ -31,13 +31,10 @@
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
 import org.apache.hadoop.io.IntWritable;
 
-@description(
-    name = "size",
-    value = "_FUNC_(a) - Returns the size of a"
-)
+@description(name = "size", value = "_FUNC_(a) - Returns the size of a")
 public class GenericUDFSize extends GenericUDF {
   private ObjectInspector returnOI;
-  private IntWritable result = new IntWritable(-1);
+  private final IntWritable result = new IntWritable(-1);
 
   @Override
   public ObjectInspector initialize(ObjectInspector[] arguments)
@@ -48,15 +45,15 @@
     }
     Category category = arguments[0].getCategory();
     String typeName = arguments[0].getTypeName();
-    if (category != Category.MAP 
-        && category != Category.LIST 
+    if (category != Category.MAP && category != Category.LIST
         && !typeName.equals(Constants.VOID_TYPE_NAME)) {
-      throw new UDFArgumentTypeException(0 ,
-          "\"" + Category.MAP.toString().toLowerCase() 
-          + "\" or \"" + Category.LIST.toString().toLowerCase()  + "\" is expected at function SIZE, "
-          + "but \"" + arguments[0].getTypeName() + "\" is found");
+      throw new UDFArgumentTypeException(0, "\""
+          + Category.MAP.toString().toLowerCase() + "\" or \""
+          + Category.LIST.toString().toLowerCase()
+          + "\" is expected at function SIZE, " + "but \""
+          + arguments[0].getTypeName() + "\" is found");
     }
-    
+
     returnOI = arguments[0];
     return PrimitiveObjectInspectorFactory.writableIntObjectInspector;
   }
@@ -64,11 +61,11 @@
   @Override
   public Object evaluate(DeferredObject[] arguments) throws HiveException {
     Object data = arguments[0].get();
-    if(returnOI.getCategory() == Category.MAP){
+    if (returnOI.getCategory() == Category.MAP) {
       result.set(((MapObjectInspector) returnOI).getMapSize(data));
-    } else if(returnOI.getCategory() == Category.LIST){
+    } else if (returnOI.getCategory() == Category.LIST) {
       result.set(((ListObjectInspector) returnOI).getListLength(data));
-    } else if(returnOI.getTypeName().equals(Constants.VOID_TYPE_NAME)) {
+    } else if (returnOI.getTypeName().equals(Constants.VOID_TYPE_NAME)) {
       // null
       result.set(-1);
     }
@@ -77,7 +74,7 @@
 
   @Override
   public String getDisplayString(String[] children) {
-    assert(children.length == 1);
+    assert (children.length == 1);
     return "size(" + children[0] + ")";
   }
 }