You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by na...@apache.org on 2009/08/14 09:48:09 UTC

svn commit: r804106 [3/24] - in /hadoop/hive/trunk: ./ contrib/src/java/org/apache/hadoop/hive/contrib/udaf/ contrib/src/java/org/apache/hadoop/hive/contrib/udaf/example/ contrib/src/java/org/apache/hadoop/hive/contrib/udf/ contrib/src/java/org/apache/...

Added: 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=804106&view=auto
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFBridge.java (added)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFBridge.java Fri Aug 14 07:48:02 2009
@@ -0,0 +1,194 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.ql.udf.generic;
+
+import java.io.Serializable;
+import java.lang.reflect.Method;
+import java.util.ArrayList;
+
+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;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDFUtils.ConversionHelper;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory.ObjectInspectorOptions;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils;
+import org.apache.hadoop.util.ReflectionUtils;
+
+/**
+ * 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.
+ * 
+ */
+public class GenericUDFBridge extends GenericUDF implements Serializable {
+
+  private static Log LOG = LogFactory.getLog(GenericUDFBridge.class.getName());
+
+  /**
+   * The name of the UDF.
+   */
+  String udfName;
+  
+  /**
+   * 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 isOperator
+   * @param udfClass
+   */
+  public GenericUDFBridge(String udfName, boolean isOperator, Class<? extends UDF> udfClass) {
+    this.udfName = udfName;
+    this.isOperator = isOperator;
+    this.udfClass = udfClass;
+  }
+
+  // For Java serialization only
+  public GenericUDFBridge() {
+  }
+  
+  public void setUdfName(String udfName) {
+    this.udfName = udfName;
+  }
+  
+  public String getUdfName() {
+    return udfName;
+  }
+  
+  public boolean isOperator() {
+    return isOperator;
+  }
+
+  public void setOperator(boolean isOperator) {
+    this.isOperator = isOperator;
+  }
+
+  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.  
+   */
+  transient ConversionHelper conversionHelper;
+  /**
+   * The actual udf object.
+   */
+  transient UDF udf;
+  /**
+   * 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);
+    
+    // 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])); 
+    }
+    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);
+    
+    return returnOI;
+  }
+
+  @Override
+  public Object evaluate(DeferredObject[] arguments) throws HiveException {
+    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));
+    
+    return result;
+  }
+
+  @Override
+  public String getDisplayString(String[] children) {
+    if (isOperator) {
+      if (children.length == 1) {
+        // Prefix operator
+        return "(" + udfName + " " + children[0] + ")";
+      } else {
+        // Infix operator
+        assert children.length == 2;
+        return "(" + children[0] + " " + udfName + " " + children[1] + ")";
+      }
+    } else {
+      StringBuilder sb = new StringBuilder();
+      sb.append(udfName);
+      sb.append("(");
+      for(int i = 0; i < children.length; i++) {
+        sb.append(children[i]);
+        if (i + 1 < children.length) {
+          sb.append(", ");
+        }
+      }
+      sb.append(")");
+      return sb.toString();
+    }
+  }
+
+}

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFUtils.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFUtils.java?rev=804106&r1=804105&r2=804106&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFUtils.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFUtils.java Fri Aug 14 07:48:02 2009
@@ -18,7 +18,10 @@
 
 package org.apache.hadoop.hive.ql.udf.generic;
 
+import java.lang.reflect.Array;
+import java.lang.reflect.GenericArrayType;
 import java.lang.reflect.Method;
+import java.lang.reflect.ParameterizedType;
 import java.lang.reflect.Type;
 import java.nio.ByteBuffer;
 import java.util.HashMap;
@@ -29,14 +32,13 @@
 import org.apache.hadoop.hive.ql.exec.UDFArgumentTypeException;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorConverters;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils;
-import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorConverters.Converter;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorConverters.IdentityConverter;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory.ObjectInspectorOptions;
 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.objectinspector.primitive.PrimitiveObjectInspectorConverter;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils;
 import org.apache.hadoop.io.Text;
@@ -172,57 +174,153 @@
   }
   
   /**
-   * Convert primitive parameters between Java and Writable when needed. 
+   * Convert parameters for the method if needed. 
    */
-  public static class PrimitiveConversionHelper {
+  public static class ConversionHelper {
 
     private Method m;
-    private ObjectInspector[] parameters;
+    private ObjectInspector[] givenParameterOIs;
+    Type[] methodParameterTypes;
+    private boolean isVariableLengthArgument; 
+    Type lastParaElementType;
     
-    Converter[] converters; 
+    boolean conversionNeeded;
+    Converter[] converters;
     Object[] convertedParameters;
+    Object[] convertedParametersInArray;
+    
+
+    private static Class<?> getClassFromType(Type t) {
+      if (t instanceof Class<?>) {
+        return (Class<?>)t;
+      } else if (t instanceof ParameterizedType) {
+        ParameterizedType pt = (ParameterizedType)t;
+        return (Class<?>)pt.getRawType();
+      }
+      return null;
+    }
     
     /**
      * Create a PrimitiveConversionHelper for Method m.  The ObjectInspector's
      * input parameters are specified in parameters.
      */
-    public PrimitiveConversionHelper(Method m, ObjectInspector[] parameters) {
+    public ConversionHelper(Method m, ObjectInspector[] parameterOIs) {
       this.m = m;
-      this.parameters = parameters;
+      this.givenParameterOIs = parameterOIs;
       
-      Type[] acceptedParameters = m.getGenericParameterTypes();
-      assert(parameters.length == acceptedParameters.length);
+      methodParameterTypes = m.getGenericParameterTypes();
+
+      // Whether the method takes an array like Object[], 
+      // or String[] etc in the last argument.
+      lastParaElementType = TypeInfoUtils.getArrayElementType(
+          methodParameterTypes.length == 0 ? null :
+          methodParameterTypes[methodParameterTypes.length-1]);
+      isVariableLengthArgument = (lastParaElementType != null);
       
-      for (int i = 0; i < parameters.length; i++) {
-        ObjectInspector acceptedParameterOI = PrimitiveObjectInspectorFactory
-            .getPrimitiveObjectInspectorFromClass((Class<?>)acceptedParameters[i]);
-        Converter pc = ObjectInspectorConverters
-            .getConverter(parameters[i], acceptedParameterOI);
-        // Conversion is needed?
-        if (pc != null) {
-          if (converters == null) {
-            // init converters only if needed.
-            converters = new Converter[parameters.length];
-            convertedParameters = new Object[parameters.length];
+      // Create the output OI array
+      ObjectInspector[] methodParameterOIs = new ObjectInspector[parameterOIs.length];
+      
+      if (isVariableLengthArgument) {
+        
+        // Copy the first methodParameterTypes.length - 1 entries
+        for (int i = 0; i < methodParameterTypes.length - 1; i++) {
+          // This method takes Object, so it accepts whatever types that are passed in.
+          if (methodParameterTypes[i] == Object.class) {
+            methodParameterOIs[i] = ObjectInspectorUtils
+                .getStandardObjectInspector(parameterOIs[i], ObjectInspectorCopyOption.JAVA);
+          } else {
+            methodParameterOIs[i] = ObjectInspectorFactory
+                .getReflectionObjectInspector(methodParameterTypes[i], ObjectInspectorOptions.JAVA);
+          }
+        }
+
+        // Deal with the last entry
+        if (lastParaElementType == Object.class) {
+          // This method takes Object[], so it accepts whatever types that are passed in.
+          for (int i = methodParameterTypes.length - 1; i < parameterOIs.length; i++) {
+            methodParameterOIs[i] = ObjectInspectorUtils
+                .getStandardObjectInspector(parameterOIs[i], ObjectInspectorCopyOption.JAVA);
+          }
+        } else {
+          // This method takes something like String[], so it only accepts something like String
+          ObjectInspector oi = ObjectInspectorFactory.getReflectionObjectInspector(
+              lastParaElementType, ObjectInspectorOptions.JAVA);
+          for (int i = methodParameterTypes.length - 1; i < parameterOIs.length; i++) {
+            methodParameterOIs[i] = oi;
+          }
+        }
+        
+      } else {
+        
+        // Normal case, the last parameter is a normal parameter.
+        assert methodParameterTypes.length == parameterOIs.length;
+        for (int i = 0; i < methodParameterTypes.length; i++) {
+          // This method takes Object, so it accepts whatever types that are passed in.
+          if (methodParameterTypes[i] == Object.class) {
+            methodParameterOIs[i] = ObjectInspectorUtils
+                .getStandardObjectInspector(parameterOIs[i], ObjectInspectorCopyOption.JAVA);
+          } else {
+            methodParameterOIs[i] = ObjectInspectorFactory
+                .getReflectionObjectInspector(methodParameterTypes[i], ObjectInspectorOptions.JAVA);
           }
-          converters[i] = pc;
         }
       }
+      
+      // Create the converters
+      conversionNeeded = false;
+      converters = new Converter[parameterOIs.length];
+      for (int i = 0; i < parameterOIs.length; i++) {
+        Converter pc = ObjectInspectorConverters
+            .getConverter(parameterOIs[i], methodParameterOIs[i]);
+        converters[i] = pc;
+        // Conversion is needed?
+        conversionNeeded = conversionNeeded || (!(pc instanceof IdentityConverter));
+      }
+      
+      if (isVariableLengthArgument) {
+        convertedParameters = new Object[methodParameterTypes.length];
+        convertedParametersInArray = (Object[])Array.newInstance(
+            getClassFromType(lastParaElementType), parameterOIs.length - methodParameterTypes.length + 1);
+        convertedParameters[convertedParameters.length - 1] = convertedParametersInArray;
+      } else {
+        convertedParameters = new Object[parameterOIs.length];
+      }
     }
     
     public Object[] convertIfNecessary(Object... parameters) {
-      if (converters == null) {
-        return parameters;
+      
+      assert(parameters.length == givenParameterOIs.length);
+      
+      if (!conversionNeeded) {
+        if (!isVariableLengthArgument) {
+          // no conversion needed, and not variable-length argument:
+          // just return what is passed in.
+          return parameters;
+        } else if (methodParameterTypes.length == 1) {
+          // no conversion needed, and variable-length argument with exact one argument
+          // just put the parameters in the array and return.
+          convertedParameters[0] = parameters;
+          return convertedParameters;
+        }
+      }
+      
+      if (isVariableLengthArgument) {
+        // convert the first methodParameterTypes.length - 1 entries
+        for (int i = 0; i < methodParameterTypes.length - 1; i++) {
+          convertedParameters[i] = converters[i].convert(parameters[i]);
+        }
+        // convert the rest and put into the last entry
+        for (int i = methodParameterTypes.length - 1; i < parameters.length; i++) {
+          convertedParametersInArray[i + 1 - methodParameterTypes.length] = 
+              converters[i].convert(parameters[i]);
+        }
       } else {
-        assert(parameters.length == convertedParameters.length);
-        for(int i = 0; i < parameters.length; i++) {
-          convertedParameters[i] =
-            converters[i] == null 
-            ? parameters[i]
-            : converters[i].convert(parameters[i]); 
+        // normal case, convert all parameters
+        for (int i = 0; i < methodParameterTypes.length; i++) {
+          convertedParameters[i] = converters[i].convert(parameters[i]);
         }
-        return convertedParameters;
       }
+      return convertedParameters;
     }
   };
 

Modified: hadoop/hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/exec/TestExecDriver.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/exec/TestExecDriver.java?rev=804106&r1=804105&r2=804106&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/exec/TestExecDriver.java (original)
+++ hadoop/hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/exec/TestExecDriver.java Fri Aug 14 07:48:02 2009
@@ -40,7 +40,7 @@
 import org.apache.hadoop.hive.ql.plan.exprNodeConstantDesc;
 import org.apache.hadoop.hive.ql.plan.exprNodeDesc;
 import org.apache.hadoop.hive.ql.plan.exprNodeFieldDesc;
-import org.apache.hadoop.hive.ql.plan.exprNodeFuncDesc;
+import org.apache.hadoop.hive.ql.plan.exprNodeGenericFuncDesc;
 import org.apache.hadoop.hive.ql.plan.extractDesc;
 import org.apache.hadoop.hive.ql.plan.fileSinkDesc;
 import org.apache.hadoop.hive.ql.plan.filterDesc;
@@ -156,31 +156,25 @@
   private filterDesc getTestFilterDesc(String column) {
     ArrayList<exprNodeDesc> children1 = new ArrayList<exprNodeDesc>();
     children1.add(new exprNodeColumnDesc(TypeInfoFactory.stringTypeInfo, column, "", false));
-    exprNodeDesc lhs = new exprNodeFuncDesc(
-        Constants.DOUBLE_TYPE_NAME,
+    exprNodeDesc lhs = new exprNodeGenericFuncDesc(
         TypeInfoFactory.doubleTypeInfo,
-        FunctionRegistry.getUDFClass(Constants.DOUBLE_TYPE_NAME),
-        FunctionRegistry.getUDFMethod(Constants.DOUBLE_TYPE_NAME, TypeInfoFactory.stringTypeInfo),
+        FunctionRegistry.getFunctionInfo(Constants.DOUBLE_TYPE_NAME).getGenericUDF(),
         children1);
     
     ArrayList<exprNodeDesc> children2 = new ArrayList<exprNodeDesc>();
     children2.add(new exprNodeConstantDesc(TypeInfoFactory.longTypeInfo, Long.valueOf(100)));
-    exprNodeDesc rhs = new exprNodeFuncDesc(
-        Constants.DOUBLE_TYPE_NAME,
+    exprNodeDesc rhs = new exprNodeGenericFuncDesc(
         TypeInfoFactory.doubleTypeInfo,
-        FunctionRegistry.getUDFClass(Constants.DOUBLE_TYPE_NAME),
-        FunctionRegistry.getUDFMethod(Constants.DOUBLE_TYPE_NAME, TypeInfoFactory.longTypeInfo),
+        FunctionRegistry.getFunctionInfo(Constants.DOUBLE_TYPE_NAME).getGenericUDF(),
         children2);
     
     ArrayList<exprNodeDesc> children3 = new ArrayList<exprNodeDesc>();
     children3.add(lhs);
     children3.add(rhs);
     
-    exprNodeDesc desc = new exprNodeFuncDesc(
-        "<",
+    exprNodeDesc desc = new exprNodeGenericFuncDesc(
         TypeInfoFactory.booleanTypeInfo,
-        FunctionRegistry.getUDFClass("<"),
-        FunctionRegistry.getUDFMethod("<", TypeInfoFactory.doubleTypeInfo, TypeInfoFactory.doubleTypeInfo),
+        FunctionRegistry.getFunctionInfo("<").getGenericUDF(),
         children3);
     
     return new filterDesc(desc, false);

Modified: hadoop/hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/exec/TestExpressionEvaluator.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/exec/TestExpressionEvaluator.java?rev=804106&r1=804105&r2=804106&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/exec/TestExpressionEvaluator.java (original)
+++ hadoop/hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/exec/TestExpressionEvaluator.java Fri Aug 14 07:48:02 2009
@@ -119,10 +119,13 @@
   }
   
   private static exprNodeDesc getListIndexNode(exprNodeDesc node, exprNodeDesc index) {
+    ArrayList<exprNodeDesc> children = new ArrayList<exprNodeDesc>(2);
+    children.add(node);
+    children.add(index);
     return new exprNodeGenericFuncDesc(
           ((ListTypeInfo)node.getTypeInfo()).getListElementTypeInfo(),
-          FunctionRegistry.getGenericUDFClassForIndex(),
-          Arrays.asList(new exprNodeDesc[]{node, index}));
+          FunctionRegistry.getGenericUDFForIndex(),
+          children);
   }
   
   public void testExprNodeFuncEvaluator() throws Throwable {

Modified: hadoop/hive/trunk/ql/src/test/results/clientpositive/cast1.q.out
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/results/clientpositive/cast1.q.out?rev=804106&r1=804105&r2=804106&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/test/results/clientpositive/cast1.q.out (original)
+++ hadoop/hive/trunk/ql/src/test/results/clientpositive/cast1.q.out Fri Aug 14 07:48:02 2009
@@ -18,19 +18,19 @@
             alias: src
             Filter Operator
               predicate:
-                  expr: (UDFToDouble(key) = UDFToDouble(86))
+                  expr: (key = 86)
                   type: boolean
               Filter Operator
                 predicate:
-                    expr: (UDFToDouble(key) = UDFToDouble(86))
+                    expr: (key = 86)
                     type: boolean
                 Select Operator
                   expressions:
                         expr: (3 + 2)
                         type: int
-                        expr: (3.0 + UDFToDouble(2))
+                        expr: (3.0 + 2)
                         type: double
-                        expr: (UDFToDouble(3) + 2.0)
+                        expr: (3 + 2.0)
                         type: double
                         expr: (3.0 + 2.0)
                         type: double
@@ -56,10 +56,10 @@
           Move Operator
             files:
                 hdfs directory: true
-                destination: file:/data/users/athusoo/commits/hive_trunk_ws1/build/ql/tmp/519069919/10000
+                destination: file:/data/users/zshao/tools/699-trunk-apache-hive/.ptest_2/build/ql/tmp/1859655774/10000
           Map Reduce
             Alias -> Map Operator Tree:
-              file:/data/users/athusoo/commits/hive_trunk_ws1/build/ql/tmp/164262834/10002 
+              file:/data/users/zshao/tools/699-trunk-apache-hive/.ptest_2/build/ql/tmp/599057975/10002 
                   Reduce Output Operator
                     sort order: 
                     Map-reduce partition columns:
@@ -108,5 +108,5 @@
 Output: default/dest1
 query: select dest1.* FROM dest1
 Input: default/dest1
-Output: file:/data/users/athusoo/commits/hive_trunk_ws1/build/ql/tmp/1857264488/10000
+Output: file:/data/users/zshao/tools/699-trunk-apache-hive/.ptest_2/build/ql/tmp/1345350120/10000
 5	5.0	5.0	5.0	5	true	1

Modified: hadoop/hive/trunk/ql/src/test/results/clientpositive/cluster.q.out
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/results/clientpositive/cluster.q.out?rev=804106&r1=804105&r2=804106&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/test/results/clientpositive/cluster.q.out (original)
+++ hadoop/hive/trunk/ql/src/test/results/clientpositive/cluster.q.out Fri Aug 14 07:48:02 2009
@@ -16,11 +16,11 @@
             alias: x
             Filter Operator
               predicate:
-                  expr: (UDFToDouble(key) = UDFToDouble(10))
+                  expr: (key = 10)
                   type: boolean
               Filter Operator
                 predicate:
-                    expr: (UDFToDouble(key) = UDFToDouble(10))
+                    expr: (key = 10)
                     type: boolean
                 Select Operator
                   expressions:
@@ -59,7 +59,7 @@
 
 query: SELECT * FROM SRC x where x.key = 10 CLUSTER BY x.key
 Input: default/src
-Output: file:/data/users/athusoo/commits/hive_trunk_ws1/build/ql/tmp/1851665085/10000
+Output: file:/data/users/zshao/tools/699-trunk-apache-hive/.ptest_0/build/ql/tmp/1039157739/10000
 10	val_10
 query: EXPLAIN
 SELECT * FROM SRC x  where x.key = 20 CLUSTER BY key
@@ -79,11 +79,11 @@
             alias: x
             Filter Operator
               predicate:
-                  expr: (UDFToDouble(key) = UDFToDouble(20))
+                  expr: (key = 20)
                   type: boolean
               Filter Operator
                 predicate:
-                    expr: (UDFToDouble(key) = UDFToDouble(20))
+                    expr: (key = 20)
                     type: boolean
                 Select Operator
                   expressions:
@@ -122,7 +122,7 @@
 
 query: SELECT * FROM SRC x where x.key = 20 CLUSTER BY key
 Input: default/src
-Output: file:/data/users/athusoo/commits/hive_trunk_ws1/build/ql/tmp/577104330/10000
+Output: file:/data/users/zshao/tools/699-trunk-apache-hive/.ptest_0/build/ql/tmp/936932129/10000
 20	val_20
 query: EXPLAIN
 SELECT x.* FROM SRC x where x.key = 20 CLUSTER BY key
@@ -142,11 +142,11 @@
             alias: x
             Filter Operator
               predicate:
-                  expr: (UDFToDouble(key) = UDFToDouble(20))
+                  expr: (key = 20)
                   type: boolean
               Filter Operator
                 predicate:
-                    expr: (UDFToDouble(key) = UDFToDouble(20))
+                    expr: (key = 20)
                     type: boolean
                 Select Operator
                   expressions:
@@ -185,7 +185,7 @@
 
 query: SELECT x.* FROM SRC x where x.key = 20 CLUSTER BY key
 Input: default/src
-Output: file:/data/users/athusoo/commits/hive_trunk_ws1/build/ql/tmp/716161226/10000
+Output: file:/data/users/zshao/tools/699-trunk-apache-hive/.ptest_0/build/ql/tmp/1777006866/10000
 20	val_20
 query: EXPLAIN
 SELECT x.*  FROM SRC x where x.key = 20 CLUSTER BY x.key
@@ -205,11 +205,11 @@
             alias: x
             Filter Operator
               predicate:
-                  expr: (UDFToDouble(key) = UDFToDouble(20))
+                  expr: (key = 20)
                   type: boolean
               Filter Operator
                 predicate:
-                    expr: (UDFToDouble(key) = UDFToDouble(20))
+                    expr: (key = 20)
                     type: boolean
                 Select Operator
                   expressions:
@@ -248,7 +248,7 @@
 
 query: SELECT x.*  FROM SRC x where x.key = 20 CLUSTER BY x.key
 Input: default/src
-Output: file:/data/users/athusoo/commits/hive_trunk_ws1/build/ql/tmp/1938918851/10000
+Output: file:/data/users/zshao/tools/699-trunk-apache-hive/.ptest_0/build/ql/tmp/332012199/10000
 20	val_20
 query: EXPLAIN
 SELECT x.key, x.value as v1 FROM SRC x where x.key = 20 CLUSTER BY key
@@ -268,11 +268,11 @@
             alias: x
             Filter Operator
               predicate:
-                  expr: (UDFToDouble(key) = UDFToDouble(20))
+                  expr: (key = 20)
                   type: boolean
               Filter Operator
                 predicate:
-                    expr: (UDFToDouble(key) = UDFToDouble(20))
+                    expr: (key = 20)
                     type: boolean
                 Select Operator
                   expressions:
@@ -311,7 +311,7 @@
 
 query: SELECT x.key, x.value as v1 FROM SRC x where x.key = 20 CLUSTER BY key
 Input: default/src
-Output: file:/data/users/athusoo/commits/hive_trunk_ws1/build/ql/tmp/111063159/10000
+Output: file:/data/users/zshao/tools/699-trunk-apache-hive/.ptest_0/build/ql/tmp/1467820372/10000
 20	val_20
 query: EXPLAIN
 SELECT x.key, x.value as v1 FROM SRC x where x.key = 20 CLUSTER BY x.key
@@ -331,11 +331,11 @@
             alias: x
             Filter Operator
               predicate:
-                  expr: (UDFToDouble(key) = UDFToDouble(20))
+                  expr: (key = 20)
                   type: boolean
               Filter Operator
                 predicate:
-                    expr: (UDFToDouble(key) = UDFToDouble(20))
+                    expr: (key = 20)
                     type: boolean
                 Select Operator
                   expressions:
@@ -374,7 +374,7 @@
 
 query: SELECT x.key, x.value as v1 FROM SRC x where x.key = 20 CLUSTER BY x.key
 Input: default/src
-Output: file:/data/users/athusoo/commits/hive_trunk_ws1/build/ql/tmp/2088645304/10000
+Output: file:/data/users/zshao/tools/699-trunk-apache-hive/.ptest_0/build/ql/tmp/1936271284/10000
 20	val_20
 query: EXPLAIN
 SELECT x.key, x.value as v1  FROM SRC x where x.key = 20 CLUSTER BY v1
@@ -394,11 +394,11 @@
             alias: x
             Filter Operator
               predicate:
-                  expr: (UDFToDouble(key) = UDFToDouble(20))
+                  expr: (key = 20)
                   type: boolean
               Filter Operator
                 predicate:
-                    expr: (UDFToDouble(key) = UDFToDouble(20))
+                    expr: (key = 20)
                     type: boolean
                 Select Operator
                   expressions:
@@ -437,7 +437,7 @@
 
 query: SELECT x.key, x.value as v1  FROM SRC x where x.key = 20 CLUSTER BY v1
 Input: default/src
-Output: file:/data/users/athusoo/commits/hive_trunk_ws1/build/ql/tmp/43650880/10000
+Output: file:/data/users/zshao/tools/699-trunk-apache-hive/.ptest_0/build/ql/tmp/554101432/10000
 20	val_20
 query: EXPLAIN
 SELECT y.* from (SELECT x.* FROM SRC x CLUSTER BY x.key) y where y.key = 20
@@ -480,7 +480,7 @@
         Extract
           Filter Operator
             predicate:
-                expr: (UDFToDouble(_col0) = UDFToDouble(20))
+                expr: (_col0 = 20)
                 type: boolean
             Select Operator
               expressions:
@@ -503,7 +503,7 @@
 
 query: SELECT y.* from (SELECT x.* FROM SRC x CLUSTER BY x.key) y where y.key = 20
 Input: default/src
-Output: file:/data/users/athusoo/commits/hive_trunk_ws1/build/ql/tmp/2050273150/10000
+Output: file:/data/users/zshao/tools/699-trunk-apache-hive/.ptest_0/build/ql/tmp/1032907487/10000
 20	val_20
 query: EXPLAIN 
 SELECT x.key, x.value as v1, y.key  FROM SRC x JOIN SRC y ON (x.key = y.key)  where x.key = 20 CLUSTER BY v1
@@ -539,7 +539,7 @@
             alias: x
             Filter Operator
               predicate:
-                  expr: (UDFToDouble(key) = UDFToDouble(20))
+                  expr: (key = 20)
                   type: boolean
               Reduce Output Operator
                 key expressions:
@@ -565,7 +565,7 @@
           outputColumnNames: _col0, _col1, _col2
           Filter Operator
             predicate:
-                expr: (UDFToDouble(_col0) = UDFToDouble(20))
+                expr: (_col0 = 20)
                 type: boolean
             Select Operator
               expressions:
@@ -586,7 +586,7 @@
   Stage: Stage-2
     Map Reduce
       Alias -> Map Operator Tree:
-        file:/data/users/athusoo/commits/hive_trunk_ws1/build/ql/tmp/568111874/10002 
+        file:/data/users/zshao/tools/699-trunk-apache-hive/.ptest_0/build/ql/tmp/2042815322/10002 
             Reduce Output Operator
               key expressions:
                     expr: _col1
@@ -619,7 +619,7 @@
 
 query: SELECT x.key, x.value as v1, y.key  FROM SRC x JOIN SRC y ON (x.key = y.key) where x.key = 20 CLUSTER BY v1
 Input: default/src
-Output: file:/data/users/athusoo/commits/hive_trunk_ws1/build/ql/tmp/1907382390/10000
+Output: file:/data/users/zshao/tools/699-trunk-apache-hive/.ptest_0/build/ql/tmp/251237187/10000
 20	val_20	20
 query: EXPLAIN 
 SELECT x.key, x.value as v1, y.*  FROM SRC x JOIN SRC y ON (x.key = y.key) where x.key = 20 CLUSTER BY v1
@@ -657,7 +657,7 @@
             alias: x
             Filter Operator
               predicate:
-                  expr: (UDFToDouble(key) = UDFToDouble(20))
+                  expr: (key = 20)
                   type: boolean
               Reduce Output Operator
                 key expressions:
@@ -683,7 +683,7 @@
           outputColumnNames: _col0, _col1, _col2, _col3
           Filter Operator
             predicate:
-                expr: (UDFToDouble(_col0) = UDFToDouble(20))
+                expr: (_col0 = 20)
                 type: boolean
             Select Operator
               expressions:
@@ -706,7 +706,7 @@
   Stage: Stage-2
     Map Reduce
       Alias -> Map Operator Tree:
-        file:/data/users/athusoo/commits/hive_trunk_ws1/build/ql/tmp/36559589/10002 
+        file:/data/users/zshao/tools/699-trunk-apache-hive/.ptest_0/build/ql/tmp/1924216417/10002 
             Reduce Output Operator
               key expressions:
                     expr: _col1
@@ -741,7 +741,7 @@
 
 query: SELECT x.key, x.value as v1, y.*  FROM SRC x JOIN SRC y ON (x.key = y.key) where x.key = 20 CLUSTER BY v1
 Input: default/src
-Output: file:/data/users/athusoo/commits/hive_trunk_ws1/build/ql/tmp/917788585/10000
+Output: file:/data/users/zshao/tools/699-trunk-apache-hive/.ptest_0/build/ql/tmp/328575061/10000
 20	val_20	20	val_20
 query: EXPLAIN
 SELECT x.key, x.value as v1, y.*  FROM SRC x JOIN SRC y ON (x.key = y.key) where x.key = 20 CLUSTER BY x.key
@@ -779,7 +779,7 @@
             alias: x
             Filter Operator
               predicate:
-                  expr: (UDFToDouble(key) = UDFToDouble(20))
+                  expr: (key = 20)
                   type: boolean
               Reduce Output Operator
                 key expressions:
@@ -805,7 +805,7 @@
           outputColumnNames: _col0, _col1, _col2, _col3
           Filter Operator
             predicate:
-                expr: (UDFToDouble(_col0) = UDFToDouble(20))
+                expr: (_col0 = 20)
                 type: boolean
             Select Operator
               expressions:
@@ -828,7 +828,7 @@
   Stage: Stage-2
     Map Reduce
       Alias -> Map Operator Tree:
-        file:/data/users/athusoo/commits/hive_trunk_ws1/build/ql/tmp/1989928864/10002 
+        file:/data/users/zshao/tools/699-trunk-apache-hive/.ptest_0/build/ql/tmp/745749935/10002 
             Reduce Output Operator
               key expressions:
                     expr: _col0
@@ -863,7 +863,7 @@
 
 query: SELECT x.key, x.value as v1, y.*  FROM SRC x JOIN SRC y ON (x.key = y.key) where x.key = 20 CLUSTER BY x.key
 Input: default/src
-Output: file:/data/users/athusoo/commits/hive_trunk_ws1/build/ql/tmp/1491329699/10000
+Output: file:/data/users/zshao/tools/699-trunk-apache-hive/.ptest_0/build/ql/tmp/66741381/10000
 20	val_20	20	val_20
 query: EXPLAIN
 SELECT x.key, x.value as v1, y.key as yk  FROM SRC x JOIN SRC y ON (x.key = y.key) where x.key = 20 CLUSTER BY key
@@ -899,7 +899,7 @@
             alias: x
             Filter Operator
               predicate:
-                  expr: (UDFToDouble(key) = UDFToDouble(20))
+                  expr: (key = 20)
                   type: boolean
               Reduce Output Operator
                 key expressions:
@@ -925,7 +925,7 @@
           outputColumnNames: _col0, _col1, _col2
           Filter Operator
             predicate:
-                expr: (UDFToDouble(_col0) = UDFToDouble(20))
+                expr: (_col0 = 20)
                 type: boolean
             Select Operator
               expressions:
@@ -946,7 +946,7 @@
   Stage: Stage-2
     Map Reduce
       Alias -> Map Operator Tree:
-        file:/data/users/athusoo/commits/hive_trunk_ws1/build/ql/tmp/1994351843/10002 
+        file:/data/users/zshao/tools/699-trunk-apache-hive/.ptest_0/build/ql/tmp/1745977063/10002 
             Reduce Output Operator
               key expressions:
                     expr: _col0
@@ -979,7 +979,7 @@
 
 query: SELECT x.key, x.value as v1, y.key as yk  FROM SRC x JOIN SRC y ON (x.key = y.key) where x.key = 20 CLUSTER BY key
 Input: default/src
-Output: file:/data/users/athusoo/commits/hive_trunk_ws1/build/ql/tmp/1415898959/10000
+Output: file:/data/users/zshao/tools/699-trunk-apache-hive/.ptest_0/build/ql/tmp/26281356/10000
 20	val_20	20
 query: EXPLAIN
 SELECT unioninput.*
@@ -1005,11 +1005,11 @@
             alias: src
             Filter Operator
               predicate:
-                  expr: (UDFToDouble(key) < UDFToDouble(100))
+                  expr: (key < 100)
                   type: boolean
               Filter Operator
                 predicate:
-                    expr: (UDFToDouble(key) < UDFToDouble(100))
+                    expr: (key < 100)
                     type: boolean
                 Select Operator
                   expressions:
@@ -1045,11 +1045,11 @@
             alias: src
             Filter Operator
               predicate:
-                  expr: (UDFToDouble(key) > UDFToDouble(100))
+                  expr: (key > 100)
                   type: boolean
               Filter Operator
                 predicate:
-                    expr: (UDFToDouble(key) > UDFToDouble(100))
+                    expr: (key > 100)
                     type: boolean
                 Select Operator
                   expressions:
@@ -1102,7 +1102,7 @@
 ) unioninput
 CLUSTER BY unioninput.key
 Input: default/src
-Output: file:/data/users/athusoo/commits/hive_trunk_ws1/build/ql/tmp/1319156082/10000
+Output: file:/data/users/zshao/tools/699-trunk-apache-hive/.ptest_0/build/ql/tmp/658971317/10000
 0	val_0
 0	val_0
 0	val_0

Modified: hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby2.q.out
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby2.q.out?rev=804106&r1=804105&r2=804106&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby2.q.out (original)
+++ hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby2.q.out Fri Aug 14 07:48:02 2009
@@ -57,7 +57,7 @@
   Stage: Stage-2
     Map Reduce
       Alias -> Map Operator Tree:
-        file:/data/users/athusoo/commits/hive_trunk_ws1/build/ql/tmp/1035591703/10002 
+        file:/data/users/zshao/tools/699-trunk-apache-hive/.ptest_3/build/ql/tmp/695763044/10002 
             Reduce Output Operator
               key expressions:
                     expr: _col0
@@ -88,7 +88,7 @@
                   type: string
                   expr: _col1
                   type: bigint
-                  expr: concat(_col0, UDFToString(_col2))
+                  expr: concat(_col0, _col2)
                   type: string
             outputColumnNames: _col0, _col1, _col2
             Select Operator
@@ -126,7 +126,7 @@
 Output: default/dest_g2
 query: SELECT dest_g2.* FROM dest_g2
 Input: default/dest_g2
-Output: file:/data/users/athusoo/commits/hive_trunk_ws1/build/ql/tmp/33506140/10000
+Output: file:/data/users/zshao/tools/699-trunk-apache-hive/.ptest_3/build/ql/tmp/347372587/10000
 0	1	00.0
 1	71	116414.0
 2	69	225571.0

Modified: hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby2_map.q.out
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby2_map.q.out?rev=804106&r1=804105&r2=804106&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby2_map.q.out (original)
+++ hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby2_map.q.out Fri Aug 14 07:48:02 2009
@@ -66,7 +66,7 @@
                   type: string
                   expr: _col1
                   type: bigint
-                  expr: concat(_col0, UDFToString(_col2))
+                  expr: concat(_col0, _col2)
                   type: string
             outputColumnNames: _col0, _col1, _col2
             Select Operator
@@ -104,7 +104,7 @@
 Output: default/dest1
 query: SELECT dest1.* FROM dest1
 Input: default/dest1
-Output: file:/data/users/athusoo/commits/hive_trunk_ws1/build/ql/tmp/907085747/10000
+Output: file:/data/users/zshao/tools/699-trunk-apache-hive/.ptest_2/build/ql/tmp/2051900086/10000
 0	1	00.0
 1	71	116414.0
 2	69	225571.0

Modified: hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby2_map_skew.q.out
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby2_map_skew.q.out?rev=804106&r1=804105&r2=804106&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby2_map_skew.q.out (original)
+++ hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby2_map_skew.q.out Fri Aug 14 07:48:02 2009
@@ -73,7 +73,7 @@
   Stage: Stage-2
     Map Reduce
       Alias -> Map Operator Tree:
-        file:/data/users/athusoo/commits/hive_trunk_ws1/build/ql/tmp/1895770841/10002 
+        file:/data/users/zshao/tools/699-trunk-apache-hive/.ptest_0/build/ql/tmp/683104031/10002 
             Reduce Output Operator
               key expressions:
                     expr: _col0
@@ -104,7 +104,7 @@
                   type: string
                   expr: _col1
                   type: bigint
-                  expr: concat(_col0, UDFToString(_col2))
+                  expr: concat(_col0, _col2)
                   type: string
             outputColumnNames: _col0, _col1, _col2
             Select Operator
@@ -142,7 +142,7 @@
 Output: default/dest1
 query: SELECT dest1.* FROM dest1
 Input: default/dest1
-Output: file:/data/users/athusoo/commits/hive_trunk_ws1/build/ql/tmp/668586222/10000
+Output: file:/data/users/zshao/tools/699-trunk-apache-hive/.ptest_0/build/ql/tmp/351974693/10000
 0	1	00.0
 1	71	116414.0
 2	69	225571.0

Modified: hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby2_noskew.q.out
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby2_noskew.q.out?rev=804106&r1=804105&r2=804106&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby2_noskew.q.out (original)
+++ hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby2_noskew.q.out Fri Aug 14 07:48:02 2009
@@ -50,7 +50,7 @@
                   type: string
                   expr: _col1
                   type: bigint
-                  expr: concat(_col0, UDFToString(_col2))
+                  expr: concat(_col0, _col2)
                   type: string
             outputColumnNames: _col0, _col1, _col2
             Select Operator
@@ -88,7 +88,7 @@
 Output: default/dest_g2
 query: SELECT dest_g2.* FROM dest_g2
 Input: default/dest_g2
-Output: file:/data/users/athusoo/commits/hive_trunk_ws1/build/ql/tmp/2078044703/10000
+Output: file:/data/users/zshao/tools/699-trunk-apache-hive/.ptest_1/build/ql/tmp/778275509/10000
 0	1	00.0
 1	71	116414.0
 2	69	225571.0

Modified: hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby_map_ppr.q.out
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby_map_ppr.q.out?rev=804106&r1=804105&r2=804106&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby_map_ppr.q.out (original)
+++ hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby_map_ppr.q.out Fri Aug 14 07:48:02 2009
@@ -65,10 +65,10 @@
                             type: double
       Needs Tagging: false
       Path -> Alias:
-        file:/data/users/athusoo/commits/hive_trunk_ws1/build/ql/test/data/warehouse/srcpart/ds=2008-04-08/hr=11 
-        file:/data/users/athusoo/commits/hive_trunk_ws1/build/ql/test/data/warehouse/srcpart/ds=2008-04-08/hr=12 
+        file:/data/users/zshao/tools/699-trunk-apache-hive/.ptest_1/build/ql/test/data/warehouse/srcpart/ds=2008-04-08/hr=11 [src]
+        file:/data/users/zshao/tools/699-trunk-apache-hive/.ptest_1/build/ql/test/data/warehouse/srcpart/ds=2008-04-08/hr=12 [src]
       Path -> Partition:
-        file:/data/users/athusoo/commits/hive_trunk_ws1/build/ql/test/data/warehouse/srcpart/ds=2008-04-08/hr=11 
+        file:/data/users/zshao/tools/699-trunk-apache-hive/.ptest_1/build/ql/test/data/warehouse/srcpart/ds=2008-04-08/hr=11 
           Partition
             partition values:
               ds 2008-04-08
@@ -87,10 +87,10 @@
                 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
                 file.inputformat org.apache.hadoop.mapred.TextInputFormat
                 file.outputformat org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-                location file:/data/users/athusoo/commits/hive_trunk_ws1/build/ql/test/data/warehouse/srcpart
+                location file:/data/users/zshao/tools/699-trunk-apache-hive/.ptest_1/build/ql/test/data/warehouse/srcpart
               serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
               name: srcpart
-        file:/data/users/athusoo/commits/hive_trunk_ws1/build/ql/test/data/warehouse/srcpart/ds=2008-04-08/hr=12 
+        file:/data/users/zshao/tools/699-trunk-apache-hive/.ptest_1/build/ql/test/data/warehouse/srcpart/ds=2008-04-08/hr=12 
           Partition
             partition values:
               ds 2008-04-08
@@ -109,7 +109,7 @@
                 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
                 file.inputformat org.apache.hadoop.mapred.TextInputFormat
                 file.outputformat org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-                location file:/data/users/athusoo/commits/hive_trunk_ws1/build/ql/test/data/warehouse/srcpart
+                location file:/data/users/zshao/tools/699-trunk-apache-hive/.ptest_1/build/ql/test/data/warehouse/srcpart
               serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
               name: srcpart
       Reduce Operator Tree:
@@ -128,7 +128,7 @@
                   type: string
                   expr: _col1
                   type: bigint
-                  expr: concat(_col0, UDFToString(_col2))
+                  expr: concat(_col0, _col2)
                   type: string
             outputColumnNames: _col0, _col1, _col2
             Select Operator
@@ -143,7 +143,7 @@
               File Output Operator
                 compressed: false
                 GlobalTableId: 1
-                directory: file:/data/users/athusoo/commits/hive_trunk_ws1/build/ql/tmp/972766884/10000
+                directory: file:/data/users/zshao/tools/699-trunk-apache-hive/.ptest_1/build/ql/tmp/1985133018/10000
                 table:
                     input format: org.apache.hadoop.mapred.TextInputFormat
                     output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
@@ -157,7 +157,7 @@
                       serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
                       file.inputformat org.apache.hadoop.mapred.TextInputFormat
                       file.outputformat org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-                      location file:/data/users/athusoo/commits/hive_trunk_ws1/build/ql/test/data/warehouse/dest1
+                      location file:/data/users/zshao/tools/699-trunk-apache-hive/.ptest_1/build/ql/test/data/warehouse/dest1
                     serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
                     name: dest1
 
@@ -165,7 +165,7 @@
     Move Operator
       tables:
           replace: true
-          source: file:/data/users/athusoo/commits/hive_trunk_ws1/build/ql/tmp/972766884/10000
+          source: file:/data/users/zshao/tools/699-trunk-apache-hive/.ptest_1/build/ql/tmp/1985133018/10000
           table:
               input format: org.apache.hadoop.mapred.TextInputFormat
               output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
@@ -179,10 +179,10 @@
                 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
                 file.inputformat org.apache.hadoop.mapred.TextInputFormat
                 file.outputformat org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-                location file:/data/users/athusoo/commits/hive_trunk_ws1/build/ql/test/data/warehouse/dest1
+                location file:/data/users/zshao/tools/699-trunk-apache-hive/.ptest_1/build/ql/test/data/warehouse/dest1
               serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
               name: dest1
-          tmp directory: file:/data/users/athusoo/commits/hive_trunk_ws1/build/ql/tmp/972766884/10001
+          tmp directory: file:/data/users/zshao/tools/699-trunk-apache-hive/.ptest_1/build/ql/tmp/1985133018/10001
 
 
 query: FROM srcpart src
@@ -195,7 +195,7 @@
 Output: default/dest1
 query: SELECT dest1.* FROM dest1
 Input: default/dest1
-Output: file:/data/users/athusoo/commits/hive_trunk_ws1/build/ql/tmp/161443012/10000
+Output: file:/data/users/zshao/tools/699-trunk-apache-hive/.ptest_1/build/ql/tmp/987556553/10000
 0	1	00.0
 1	71	132828.0
 2	69	251142.0

Modified: hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby_ppr.q.out
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby_ppr.q.out?rev=804106&r1=804105&r2=804106&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby_ppr.q.out (original)
+++ hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby_ppr.q.out Fri Aug 14 07:48:02 2009
@@ -49,10 +49,10 @@
                     tag: -1
       Needs Tagging: false
       Path -> Alias:
-        file:/data/users/athusoo/commits/hive_trunk_ws1/build/ql/test/data/warehouse/srcpart/ds=2008-04-08/hr=11 
-        file:/data/users/athusoo/commits/hive_trunk_ws1/build/ql/test/data/warehouse/srcpart/ds=2008-04-08/hr=12 
+        file:/data/users/zshao/tools/699-trunk-apache-hive/.ptest_0/build/ql/test/data/warehouse/srcpart/ds=2008-04-08/hr=11 [src]
+        file:/data/users/zshao/tools/699-trunk-apache-hive/.ptest_0/build/ql/test/data/warehouse/srcpart/ds=2008-04-08/hr=12 [src]
       Path -> Partition:
-        file:/data/users/athusoo/commits/hive_trunk_ws1/build/ql/test/data/warehouse/srcpart/ds=2008-04-08/hr=11 
+        file:/data/users/zshao/tools/699-trunk-apache-hive/.ptest_0/build/ql/test/data/warehouse/srcpart/ds=2008-04-08/hr=11 
           Partition
             partition values:
               ds 2008-04-08
@@ -71,10 +71,10 @@
                 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
                 file.inputformat org.apache.hadoop.mapred.TextInputFormat
                 file.outputformat org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-                location file:/data/users/athusoo/commits/hive_trunk_ws1/build/ql/test/data/warehouse/srcpart
+                location file:/data/users/zshao/tools/699-trunk-apache-hive/.ptest_0/build/ql/test/data/warehouse/srcpart
               serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
               name: srcpart
-        file:/data/users/athusoo/commits/hive_trunk_ws1/build/ql/test/data/warehouse/srcpart/ds=2008-04-08/hr=12 
+        file:/data/users/zshao/tools/699-trunk-apache-hive/.ptest_0/build/ql/test/data/warehouse/srcpart/ds=2008-04-08/hr=12 
           Partition
             partition values:
               ds 2008-04-08
@@ -93,7 +93,7 @@
                 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
                 file.inputformat org.apache.hadoop.mapred.TextInputFormat
                 file.outputformat org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-                location file:/data/users/athusoo/commits/hive_trunk_ws1/build/ql/test/data/warehouse/srcpart
+                location file:/data/users/zshao/tools/699-trunk-apache-hive/.ptest_0/build/ql/test/data/warehouse/srcpart
               serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
               name: srcpart
       Reduce Operator Tree:
@@ -112,7 +112,7 @@
                   type: string
                   expr: _col1
                   type: bigint
-                  expr: concat(_col0, UDFToString(_col2))
+                  expr: concat(_col0, _col2)
                   type: string
             outputColumnNames: _col0, _col1, _col2
             Select Operator
@@ -127,7 +127,7 @@
               File Output Operator
                 compressed: false
                 GlobalTableId: 1
-                directory: file:/data/users/athusoo/commits/hive_trunk_ws1/build/ql/tmp/633188587/10000
+                directory: file:/data/users/zshao/tools/699-trunk-apache-hive/.ptest_0/build/ql/tmp/1395781229/10000
                 table:
                     input format: org.apache.hadoop.mapred.TextInputFormat
                     output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
@@ -141,7 +141,7 @@
                       serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
                       file.inputformat org.apache.hadoop.mapred.TextInputFormat
                       file.outputformat org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-                      location file:/data/users/athusoo/commits/hive_trunk_ws1/build/ql/test/data/warehouse/dest1
+                      location file:/data/users/zshao/tools/699-trunk-apache-hive/.ptest_0/build/ql/test/data/warehouse/dest1
                     serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
                     name: dest1
 
@@ -149,7 +149,7 @@
     Move Operator
       tables:
           replace: true
-          source: file:/data/users/athusoo/commits/hive_trunk_ws1/build/ql/tmp/633188587/10000
+          source: file:/data/users/zshao/tools/699-trunk-apache-hive/.ptest_0/build/ql/tmp/1395781229/10000
           table:
               input format: org.apache.hadoop.mapred.TextInputFormat
               output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
@@ -163,10 +163,10 @@
                 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
                 file.inputformat org.apache.hadoop.mapred.TextInputFormat
                 file.outputformat org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-                location file:/data/users/athusoo/commits/hive_trunk_ws1/build/ql/test/data/warehouse/dest1
+                location file:/data/users/zshao/tools/699-trunk-apache-hive/.ptest_0/build/ql/test/data/warehouse/dest1
               serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
               name: dest1
-          tmp directory: file:/data/users/athusoo/commits/hive_trunk_ws1/build/ql/tmp/633188587/10001
+          tmp directory: file:/data/users/zshao/tools/699-trunk-apache-hive/.ptest_0/build/ql/tmp/1395781229/10001
 
 
 query: FROM srcpart src
@@ -179,7 +179,7 @@
 Output: default/dest1
 query: SELECT dest1.* FROM dest1
 Input: default/dest1
-Output: file:/data/users/athusoo/commits/hive_trunk_ws1/build/ql/tmp/561326404/10000
+Output: file:/data/users/zshao/tools/699-trunk-apache-hive/.ptest_0/build/ql/tmp/1259627216/10000
 0	1	00.0
 1	71	132828.0
 2	69	251142.0

Modified: hadoop/hive/trunk/ql/src/test/results/clientpositive/implicit_cast1.q.out
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/results/clientpositive/implicit_cast1.q.out?rev=804106&r1=804105&r2=804106&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/test/results/clientpositive/implicit_cast1.q.out (original)
+++ hadoop/hive/trunk/ql/src/test/results/clientpositive/implicit_cast1.q.out Fri Aug 14 07:48:02 2009
@@ -19,11 +19,11 @@
             alias: implicit_test1
             Filter Operator
               predicate:
-                  expr: (UDFToDouble(a) <> UDFToDouble(0))
+                  expr: (a <> 0)
                   type: boolean
               Filter Operator
                 predicate:
-                    expr: (UDFToDouble(a) <> UDFToDouble(0))
+                    expr: (a <> 0)
                     type: boolean
                 Select Operator
                   expressions:
@@ -48,5 +48,5 @@
 FROM implicit_test1
 WHERE implicit_test1.a <> 0
 Input: default/implicit_test1
-Output: file:/data/users/athusoo/commits/hive_trunk_ws1/build/ql/tmp/915534198/10000
+Output: file:/data/users/zshao/tools/699-trunk-apache-hive/.ptest_1/build/ql/tmp/1171789993/10000
 query: DROP TABLE implicit_test1

Modified: hadoop/hive/trunk/ql/src/test/results/clientpositive/input11.q.out
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/results/clientpositive/input11.q.out?rev=804106&r1=804105&r2=804106&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/test/results/clientpositive/input11.q.out (original)
+++ hadoop/hive/trunk/ql/src/test/results/clientpositive/input11.q.out Fri Aug 14 07:48:02 2009
@@ -19,11 +19,11 @@
             alias: src
             Filter Operator
               predicate:
-                  expr: (UDFToDouble(key) < UDFToDouble(100))
+                  expr: (key < 100)
                   type: boolean
               Filter Operator
                 predicate:
-                    expr: (UDFToDouble(key) < UDFToDouble(100))
+                    expr: (key < 100)
                     type: boolean
                 Select Operator
                   expressions:
@@ -54,10 +54,10 @@
           Move Operator
             files:
                 hdfs directory: true
-                destination: file:/data/users/athusoo/commits/hive_trunk_ws1/build/ql/tmp/1030796464/10000
+                destination: file:/data/users/zshao/tools/699-trunk-apache-hive/.ptest_2/build/ql/tmp/880580801/10000
           Map Reduce
             Alias -> Map Operator Tree:
-              file:/data/users/athusoo/commits/hive_trunk_ws1/build/ql/tmp/1121239779/10002 
+              file:/data/users/zshao/tools/699-trunk-apache-hive/.ptest_2/build/ql/tmp/1924406901/10002 
                   Reduce Output Operator
                     sort order: 
                     Map-reduce partition columns:
@@ -97,7 +97,7 @@
 Output: default/dest1
 query: SELECT dest1.* FROM dest1
 Input: default/dest1
-Output: file:/data/users/athusoo/commits/hive_trunk_ws1/build/ql/tmp/2019052885/10000
+Output: file:/data/users/zshao/tools/699-trunk-apache-hive/.ptest_2/build/ql/tmp/1414636145/10000
 86	val_86
 27	val_27
 98	val_98

Modified: hadoop/hive/trunk/ql/src/test/results/clientpositive/input11_limit.q.out
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/results/clientpositive/input11_limit.q.out?rev=804106&r1=804105&r2=804106&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/test/results/clientpositive/input11_limit.q.out (original)
+++ hadoop/hive/trunk/ql/src/test/results/clientpositive/input11_limit.q.out Fri Aug 14 07:48:02 2009
@@ -18,11 +18,11 @@
             alias: src
             Filter Operator
               predicate:
-                  expr: (UDFToDouble(key) < UDFToDouble(100))
+                  expr: (key < 100)
                   type: boolean
               Filter Operator
                 predicate:
-                    expr: (UDFToDouble(key) < UDFToDouble(100))
+                    expr: (key < 100)
                     type: boolean
                 Select Operator
                   expressions:
@@ -76,7 +76,7 @@
 Output: default/dest1
 query: SELECT dest1.* FROM dest1
 Input: default/dest1
-Output: file:/data/users/athusoo/commits/hive_trunk_ws1/build/ql/tmp/1979379952/10000
+Output: file:/data/users/zshao/tools/699-trunk-apache-hive/.ptest_1/build/ql/tmp/232444708/10000
 86	val_86
 27	val_27
 98	val_98

Modified: hadoop/hive/trunk/ql/src/test/results/clientpositive/input12.q.out
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/results/clientpositive/input12.q.out?rev=804106&r1=804105&r2=804106&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/test/results/clientpositive/input12.q.out (original)
+++ hadoop/hive/trunk/ql/src/test/results/clientpositive/input12.q.out Fri Aug 14 07:48:02 2009
@@ -27,7 +27,7 @@
             alias: src
             Filter Operator
               predicate:
-                  expr: (UDFToDouble(key) < UDFToDouble(100))
+                  expr: (key < 100)
                   type: boolean
               Select Operator
                 expressions:
@@ -53,7 +53,7 @@
                         name: dest1
             Filter Operator
               predicate:
-                  expr: ((UDFToDouble(key) >= UDFToDouble(100)) and (UDFToDouble(key) < UDFToDouble(200)))
+                  expr: ((key >= 100) and (key < 200))
                   type: boolean
               Select Operator
                 expressions:
@@ -79,7 +79,7 @@
                         name: dest2
             Filter Operator
               predicate:
-                  expr: (UDFToDouble(key) >= UDFToDouble(200))
+                  expr: (key >= 200)
                   type: boolean
               Select Operator
                 expressions:
@@ -106,10 +106,10 @@
           Move Operator
             files:
                 hdfs directory: true
-                destination: file:/data/users/athusoo/commits/hive_trunk_ws1/build/ql/tmp/1924563025/10000
+                destination: file:/data/users/zshao/tools/699-trunk-apache-hive/.ptest_0/build/ql/tmp/1422121062/10000
           Map Reduce
             Alias -> Map Operator Tree:
-              file:/data/users/athusoo/commits/hive_trunk_ws1/build/ql/tmp/467218372/10006 
+              file:/data/users/zshao/tools/699-trunk-apache-hive/.ptest_0/build/ql/tmp/1839749098/10006 
                   Reduce Output Operator
                     sort order: 
                     Map-reduce partition columns:
@@ -148,10 +148,10 @@
           Move Operator
             files:
                 hdfs directory: true
-                destination: file:/data/users/athusoo/commits/hive_trunk_ws1/build/ql/tmp/1924563025/10002
+                destination: file:/data/users/zshao/tools/699-trunk-apache-hive/.ptest_0/build/ql/tmp/1422121062/10002
           Map Reduce
             Alias -> Map Operator Tree:
-              file:/data/users/athusoo/commits/hive_trunk_ws1/build/ql/tmp/467218372/10007 
+              file:/data/users/zshao/tools/699-trunk-apache-hive/.ptest_0/build/ql/tmp/1839749098/10007 
                   Reduce Output Operator
                     sort order: 
                     Map-reduce partition columns:
@@ -190,10 +190,10 @@
           Move Operator
             files:
                 hdfs directory: true
-                destination: file:/data/users/athusoo/commits/hive_trunk_ws1/build/ql/tmp/1924563025/10004
+                destination: file:/data/users/zshao/tools/699-trunk-apache-hive/.ptest_0/build/ql/tmp/1422121062/10004
           Map Reduce
             Alias -> Map Operator Tree:
-              file:/data/users/athusoo/commits/hive_trunk_ws1/build/ql/tmp/467218372/10008 
+              file:/data/users/zshao/tools/699-trunk-apache-hive/.ptest_0/build/ql/tmp/1839749098/10008 
                   Reduce Output Operator
                     sort order: 
                     Map-reduce partition columns:
@@ -238,7 +238,7 @@
 Output: default/dest3/ds=2008-04-08/hr=12
 query: SELECT dest1.* FROM dest1
 Input: default/dest1
-Output: file:/data/users/athusoo/commits/hive_trunk_ws1/build/ql/tmp/1597762873/10000
+Output: file:/data/users/zshao/tools/699-trunk-apache-hive/.ptest_0/build/ql/tmp/1815199816/10000
 86	val_86
 27	val_27
 98	val_98
@@ -325,7 +325,7 @@
 97	val_97
 query: SELECT dest2.* FROM dest2
 Input: default/dest2
-Output: file:/data/users/athusoo/commits/hive_trunk_ws1/build/ql/tmp/106250860/10000
+Output: file:/data/users/zshao/tools/699-trunk-apache-hive/.ptest_0/build/ql/tmp/1685078450/10000
 165	val_165
 193	val_193
 150	val_150
@@ -433,7 +433,7 @@
 169	val_169
 query: SELECT dest3.* FROM dest3
 Input: default/dest3/ds=2008-04-08/hr=12
-Output: file:/data/users/athusoo/commits/hive_trunk_ws1/build/ql/tmp/852643259/10000
+Output: file:/data/users/zshao/tools/699-trunk-apache-hive/.ptest_0/build/ql/tmp/1466798382/10000
 238	2008-04-08	12
 311	2008-04-08	12
 409	2008-04-08	12

Modified: hadoop/hive/trunk/ql/src/test/results/clientpositive/input13.q.out
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/results/clientpositive/input13.q.out?rev=804106&r1=804105&r2=804106&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/test/results/clientpositive/input13.q.out (original)
+++ hadoop/hive/trunk/ql/src/test/results/clientpositive/input13.q.out Fri Aug 14 07:48:02 2009
@@ -30,7 +30,7 @@
             alias: src
             Filter Operator
               predicate:
-                  expr: (UDFToDouble(key) < UDFToDouble(100))
+                  expr: (key < 100)
                   type: boolean
               Select Operator
                 expressions:
@@ -56,7 +56,7 @@
                         name: dest1
             Filter Operator
               predicate:
-                  expr: ((UDFToDouble(key) >= UDFToDouble(100)) and (UDFToDouble(key) < UDFToDouble(200)))
+                  expr: ((key >= 100) and (key < 200))
                   type: boolean
               Select Operator
                 expressions:
@@ -82,7 +82,7 @@
                         name: dest2
             Filter Operator
               predicate:
-                  expr: ((UDFToDouble(key) >= UDFToDouble(200)) and (UDFToDouble(key) < UDFToDouble(300)))
+                  expr: ((key >= 200) and (key < 300))
                   type: boolean
               Select Operator
                 expressions:
@@ -104,7 +104,7 @@
                         name: dest3
             Filter Operator
               predicate:
-                  expr: (UDFToDouble(key) >= UDFToDouble(300))
+                  expr: (key >= 300)
                   type: boolean
               Select Operator
                 expressions:
@@ -124,10 +124,10 @@
           Move Operator
             files:
                 hdfs directory: true
-                destination: file:/data/users/athusoo/commits/hive_trunk_ws1/build/ql/tmp/670802827/10000
+                destination: file:/data/users/zshao/tools/699-trunk-apache-hive/.ptest_1/build/ql/tmp/2000665418/10000
           Map Reduce
             Alias -> Map Operator Tree:
-              file:/data/users/athusoo/commits/hive_trunk_ws1/build/ql/tmp/1557476151/10007 
+              file:/data/users/zshao/tools/699-trunk-apache-hive/.ptest_1/build/ql/tmp/1057074180/10007 
                   Reduce Output Operator
                     sort order: 
                     Map-reduce partition columns:
@@ -166,10 +166,10 @@
           Move Operator
             files:
                 hdfs directory: true
-                destination: file:/data/users/athusoo/commits/hive_trunk_ws1/build/ql/tmp/670802827/10002
+                destination: file:/data/users/zshao/tools/699-trunk-apache-hive/.ptest_1/build/ql/tmp/2000665418/10002
           Map Reduce
             Alias -> Map Operator Tree:
-              file:/data/users/athusoo/commits/hive_trunk_ws1/build/ql/tmp/1557476151/10008 
+              file:/data/users/zshao/tools/699-trunk-apache-hive/.ptest_1/build/ql/tmp/1057074180/10008 
                   Reduce Output Operator
                     sort order: 
                     Map-reduce partition columns:
@@ -208,10 +208,10 @@
           Move Operator
             files:
                 hdfs directory: true
-                destination: file:/data/users/athusoo/commits/hive_trunk_ws1/build/ql/tmp/670802827/10004
+                destination: file:/data/users/zshao/tools/699-trunk-apache-hive/.ptest_1/build/ql/tmp/2000665418/10004
           Map Reduce
             Alias -> Map Operator Tree:
-              file:/data/users/athusoo/commits/hive_trunk_ws1/build/ql/tmp/1557476151/10009 
+              file:/data/users/zshao/tools/699-trunk-apache-hive/.ptest_1/build/ql/tmp/1057074180/10009 
                   Reduce Output Operator
                     sort order: 
                     Map-reduce partition columns:
@@ -251,10 +251,10 @@
           Move Operator
             files:
                 hdfs directory: true
-                destination: file:/data/users/athusoo/commits/hive_trunk_ws1/build/ql/tmp/670802827/10006
+                destination: file:/data/users/zshao/tools/699-trunk-apache-hive/.ptest_1/build/ql/tmp/2000665418/10006
           Map Reduce
             Alias -> Map Operator Tree:
-              file:/data/users/athusoo/commits/hive_trunk_ws1/build/ql/tmp/1557476151/10010 
+              file:/data/users/zshao/tools/699-trunk-apache-hive/.ptest_1/build/ql/tmp/1057074180/10010 
                   Reduce Output Operator
                     sort order: 
                     Map-reduce partition columns:
@@ -292,7 +292,7 @@
 Output: ../build/ql/test/data/warehouse/dest4.out
 query: SELECT dest1.* FROM dest1
 Input: default/dest1
-Output: file:/data/users/athusoo/commits/hive_trunk_ws1/build/ql/tmp/881401051/10000
+Output: file:/data/users/zshao/tools/699-trunk-apache-hive/.ptest_1/build/ql/tmp/920411544/10000
 86	val_86
 27	val_27
 98	val_98
@@ -379,7 +379,7 @@
 97	val_97
 query: SELECT dest2.* FROM dest2
 Input: default/dest2
-Output: file:/data/users/athusoo/commits/hive_trunk_ws1/build/ql/tmp/2139886551/10000
+Output: file:/data/users/zshao/tools/699-trunk-apache-hive/.ptest_1/build/ql/tmp/108447596/10000
 165	val_165
 193	val_193
 150	val_150
@@ -487,7 +487,7 @@
 169	val_169
 query: SELECT dest3.* FROM dest3
 Input: default/dest3/ds=2008-04-08/hr=12
-Output: file:/data/users/athusoo/commits/hive_trunk_ws1/build/ql/tmp/1615208828/10000
+Output: file:/data/users/zshao/tools/699-trunk-apache-hive/.ptest_1/build/ql/tmp/245976793/10000
 238	2008-04-08	12
 255	2008-04-08	12
 278	2008-04-08	12

Modified: hadoop/hive/trunk/ql/src/test/results/clientpositive/input14.q.out
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/results/clientpositive/input14.q.out?rev=804106&r1=804105&r2=804106&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/test/results/clientpositive/input14.q.out (original)
+++ hadoop/hive/trunk/ql/src/test/results/clientpositive/input14.q.out Fri Aug 14 07:48:02 2009
@@ -51,7 +51,7 @@
         Extract
           Filter Operator
             predicate:
-                expr: (UDFToDouble(_col0) < UDFToDouble(100))
+                expr: (_col0 < 100)
                 type: boolean
             Select Operator
               expressions:
@@ -98,7 +98,7 @@
 Output: default/dest1
 query: SELECT dest1.* FROM dest1
 Input: default/dest1
-Output: file:/data/users/athusoo/commits/hive_trunk_ws1/build/ql/tmp/1767626690/10000
+Output: file:/data/users/zshao/tools/699-trunk-apache-hive/.ptest_0/build/ql/tmp/547922679/10000
 0	val_0
 0	val_0
 0	val_0

Modified: hadoop/hive/trunk/ql/src/test/results/clientpositive/input14_limit.q.out
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/results/clientpositive/input14_limit.q.out?rev=804106&r1=804105&r2=804106&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/test/results/clientpositive/input14_limit.q.out (original)
+++ hadoop/hive/trunk/ql/src/test/results/clientpositive/input14_limit.q.out Fri Aug 14 07:48:02 2009
@@ -61,7 +61,7 @@
   Stage: Stage-2
     Map Reduce
       Alias -> Map Operator Tree:
-        file:/data/users/athusoo/commits/hive_trunk_ws1/build/ql/tmp/34766397/10002 
+        file:/data/users/zshao/tools/699-trunk-apache-hive/.ptest_2/build/ql/tmp/293162762/10002 
             Reduce Output Operator
               key expressions:
                     expr: _col0
@@ -81,7 +81,7 @@
           Limit
             Filter Operator
               predicate:
-                  expr: (UDFToDouble(_col0) < UDFToDouble(100))
+                  expr: (_col0 < 100)
                   type: boolean
               Select Operator
                 expressions:
@@ -128,7 +128,7 @@
 Output: default/dest1
 query: SELECT dest1.* FROM dest1
 Input: default/dest1
-Output: file:/data/users/athusoo/commits/hive_trunk_ws1/build/ql/tmp/452755876/10000
+Output: file:/data/users/zshao/tools/699-trunk-apache-hive/.ptest_2/build/ql/tmp/181640407/10000
 0	val_0
 0	val_0
 0	val_0

Modified: hadoop/hive/trunk/ql/src/test/results/clientpositive/input18.q.out
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/results/clientpositive/input18.q.out?rev=804106&r1=804105&r2=804106&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/test/results/clientpositive/input18.q.out (original)
+++ hadoop/hive/trunk/ql/src/test/results/clientpositive/input18.q.out Fri Aug 14 07:48:02 2009
@@ -55,7 +55,7 @@
         Extract
           Filter Operator
             predicate:
-                expr: (UDFToDouble(_col0) < UDFToDouble(100))
+                expr: (_col0 < 100)
                 type: boolean
             Select Operator
               expressions:
@@ -102,7 +102,7 @@
 Output: default/dest1
 query: SELECT dest1.* FROM dest1
 Input: default/dest1
-Output: file:/data/users/athusoo/commits/hive_trunk_ws1/build/ql/tmp/1817619374/10000
+Output: file:/data/users/zshao/tools/699-trunk-apache-hive/.ptest_2/build/ql/tmp/1006587831/10000
 0	val_0+3+7
 0	val_0+3+7
 0	val_0+3+7

Modified: hadoop/hive/trunk/ql/src/test/results/clientpositive/input1_limit.q.out
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/results/clientpositive/input1_limit.q.out?rev=804106&r1=804105&r2=804106&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/test/results/clientpositive/input1_limit.q.out (original)
+++ hadoop/hive/trunk/ql/src/test/results/clientpositive/input1_limit.q.out Fri Aug 14 07:48:02 2009
@@ -22,7 +22,7 @@
             alias: src
             Filter Operator
               predicate:
-                  expr: (UDFToDouble(key) < UDFToDouble(100))
+                  expr: (key < 100)
                   type: boolean
               Select Operator
                 expressions:
@@ -42,7 +42,7 @@
                           type: string
             Filter Operator
               predicate:
-                  expr: (UDFToDouble(key) < UDFToDouble(100))
+                  expr: (key < 100)
                   type: boolean
               Select Operator
                 expressions:
@@ -90,7 +90,7 @@
   Stage: Stage-3
     Map Reduce
       Alias -> Map Operator Tree:
-        file:/data/users/athusoo/commits/hive_trunk_ws1/build/ql/tmp/1614539331/10004 
+        file:/data/users/zshao/tools/699-trunk-apache-hive/.ptest_0/build/ql/tmp/1991354827/10004 
             Reduce Output Operator
               sort order: 
               tag: -1
@@ -137,7 +137,7 @@
 Output: default/dest2
 query: SELECT dest1.* FROM dest1
 Input: default/dest1
-Output: file:/data/users/athusoo/commits/hive_trunk_ws1/build/ql/tmp/1722063757/10000
+Output: file:/data/users/zshao/tools/699-trunk-apache-hive/.ptest_0/build/ql/tmp/1070027433/10000
 86	val_86
 27	val_27
 98	val_98
@@ -150,7 +150,7 @@
 57	val_57
 query: SELECT dest2.* FROM dest2
 Input: default/dest2
-Output: file:/data/users/athusoo/commits/hive_trunk_ws1/build/ql/tmp/1675468164/10000
+Output: file:/data/users/zshao/tools/699-trunk-apache-hive/.ptest_0/build/ql/tmp/1883602206/10000
 86	val_86
 27	val_27
 98	val_98

Modified: hadoop/hive/trunk/ql/src/test/results/clientpositive/input2_limit.q.out
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/results/clientpositive/input2_limit.q.out?rev=804106&r1=804105&r2=804106&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/test/results/clientpositive/input2_limit.q.out (original)
+++ hadoop/hive/trunk/ql/src/test/results/clientpositive/input2_limit.q.out Fri Aug 14 07:48:02 2009
@@ -16,11 +16,11 @@
             alias: x
             Filter Operator
               predicate:
-                  expr: (UDFToDouble(key) < UDFToDouble(300))
+                  expr: (key < 300)
                   type: boolean
               Filter Operator
                 predicate:
-                    expr: (UDFToDouble(key) < UDFToDouble(300))
+                    expr: (key < 300)
                     type: boolean
                 Select Operator
                   expressions:
@@ -44,7 +44,7 @@
 
 query: SELECT x.* FROM SRC x WHERE x.key < 300 LIMIT 5
 Input: default/src
-Output: file:/data/users/athusoo/commits/hive_trunk_ws1/build/ql/tmp/515715119/10000
+Output: file:/data/users/zshao/tools/699-trunk-apache-hive/.ptest_0/build/ql/tmp/268148023/10000
 238	val_238
 86	val_86
 27	val_27

Modified: hadoop/hive/trunk/ql/src/test/results/clientpositive/input30.q.out
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/results/clientpositive/input30.q.out?rev=804106&r1=804105&r2=804106&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/test/results/clientpositive/input30.q.out (original)
+++ hadoop/hive/trunk/ql/src/test/results/clientpositive/input30.q.out Fri Aug 14 07:48:02 2009
@@ -21,7 +21,7 @@
             alias: src
             Filter Operator
               predicate:
-                  expr: (((hash(rand(UDFToLong(460476415))) & 2147483647) % 32) = 0)
+                  expr: (((hash(rand(460476415)) & 2147483647) % 32) = 0)
                   type: boolean
               Select Operator
                 Group By Operator
@@ -77,7 +77,7 @@
 Output: default/tst_dest30
 query: select * from tst_dest30
 Input: default/tst_dest30
-Output: file:/data/users/athusoo/commits/hive_trunk_ws1/build/ql/tmp/1316741609/10000
+Output: file:/data/users/zshao/tools/699-trunk-apache-hive/.ptest_2/build/ql/tmp/606603037/10000
 18
 query: drop table tst_dest30
 query: drop table dest30

Modified: hadoop/hive/trunk/ql/src/test/results/clientpositive/input8.q.out
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/results/clientpositive/input8.q.out?rev=804106&r1=804105&r2=804106&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/test/results/clientpositive/input8.q.out (original)
+++ hadoop/hive/trunk/ql/src/test/results/clientpositive/input8.q.out Fri Aug 14 07:48:02 2009
@@ -19,9 +19,9 @@
             alias: src1
             Select Operator
               expressions:
-                    expr: (UDFToDouble(4) + null)
+                    expr: (4 + null)
                     type: double
-                    expr: (UDFToDouble(key) - null)
+                    expr: (key - null)
                     type: double
                     expr: (null + null)
                     type: double
@@ -50,10 +50,10 @@
           Move Operator
             files:
                 hdfs directory: true
-                destination: file:/data/users/athusoo/commits/hive_trunk_ws1/build/ql/tmp/1626829975/10000
+                destination: file:/data/users/zshao/tools/699-trunk-apache-hive/.ptest_2/build/ql/tmp/124487201/10000
           Map Reduce
             Alias -> Map Operator Tree:
-              file:/data/users/athusoo/commits/hive_trunk_ws1/build/ql/tmp/2078440366/10002 
+              file:/data/users/zshao/tools/699-trunk-apache-hive/.ptest_2/build/ql/tmp/473256518/10002 
                   Reduce Output Operator
                     sort order: 
                     Map-reduce partition columns:
@@ -95,7 +95,7 @@
 Output: default/dest1
 query: SELECT dest1.* FROM dest1
 Input: default/dest1
-Output: file:/data/users/athusoo/commits/hive_trunk_ws1/build/ql/tmp/60632490/10000
+Output: file:/data/users/zshao/tools/699-trunk-apache-hive/.ptest_2/build/ql/tmp/963034629/10000
 NULL	NULL	NULL
 NULL	NULL	NULL
 NULL	NULL	NULL

Modified: hadoop/hive/trunk/ql/src/test/results/clientpositive/input_part1.q.out
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/results/clientpositive/input_part1.q.out?rev=804106&r1=804105&r2=804106&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/test/results/clientpositive/input_part1.q.out (original)
+++ hadoop/hive/trunk/ql/src/test/results/clientpositive/input_part1.q.out Fri Aug 14 07:48:02 2009
@@ -20,12 +20,12 @@
             Filter Operator
               isSamplingPred: false
               predicate:
-                  expr: (((UDFToDouble(key) < UDFToDouble(100)) and (ds = '2008-04-08')) and (hr = '12'))
+                  expr: (((key < 100) and (ds = '2008-04-08')) and (hr = '12'))
                   type: boolean
               Filter Operator
                 isSamplingPred: false
                 predicate:
-                    expr: (((UDFToDouble(key) < UDFToDouble(100)) and (ds = '2008-04-08')) and (hr = '12'))
+                    expr: (((key < 100) and (ds = '2008-04-08')) and (hr = '12'))
                     type: boolean
                 Select Operator
                   expressions:
@@ -52,7 +52,7 @@
                     File Output Operator
                       compressed: false
                       GlobalTableId: 1
-                      directory: file:/data/users/athusoo/commits/hive_trunk_ws1/build/ql/tmp/430187082/10002
+                      directory: file:/data/users/zshao/tools/699-trunk-apache-hive/.ptest_0/build/ql/tmp/1871117581/10002
                       table:
                           input format: org.apache.hadoop.mapred.TextInputFormat
                           output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
@@ -66,14 +66,14 @@
                             serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
                             file.inputformat org.apache.hadoop.mapred.TextInputFormat
                             file.outputformat org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-                            location file:/data/users/athusoo/commits/hive_trunk_ws1/build/ql/test/data/warehouse/dest1
+                            location file:/data/users/zshao/tools/699-trunk-apache-hive/.ptest_0/build/ql/test/data/warehouse/dest1
                           serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
                           name: dest1
       Needs Tagging: false
       Path -> Alias:
-        file:/data/users/athusoo/commits/hive_trunk_ws1/build/ql/test/data/warehouse/srcpart/ds=2008-04-08/hr=12 
+        file:/data/users/zshao/tools/699-trunk-apache-hive/.ptest_0/build/ql/test/data/warehouse/srcpart/ds=2008-04-08/hr=12 [srcpart]
       Path -> Partition:
-        file:/data/users/athusoo/commits/hive_trunk_ws1/build/ql/test/data/warehouse/srcpart/ds=2008-04-08/hr=12 
+        file:/data/users/zshao/tools/699-trunk-apache-hive/.ptest_0/build/ql/test/data/warehouse/srcpart/ds=2008-04-08/hr=12 
           Partition
             partition values:
               ds 2008-04-08
@@ -92,7 +92,7 @@
                 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
                 file.inputformat org.apache.hadoop.mapred.TextInputFormat
                 file.outputformat org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-                location file:/data/users/athusoo/commits/hive_trunk_ws1/build/ql/test/data/warehouse/srcpart
+                location file:/data/users/zshao/tools/699-trunk-apache-hive/.ptest_0/build/ql/test/data/warehouse/srcpart
               serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
               name: srcpart
 
@@ -102,11 +102,11 @@
           Move Operator
             files:
                 hdfs directory: true
-                source: file:/data/users/athusoo/commits/hive_trunk_ws1/build/ql/tmp/430187082/10002
-                destination: file:/data/users/athusoo/commits/hive_trunk_ws1/build/ql/tmp/1548094656/10000
+                source: file:/data/users/zshao/tools/699-trunk-apache-hive/.ptest_0/build/ql/tmp/1871117581/10002
+                destination: file:/data/users/zshao/tools/699-trunk-apache-hive/.ptest_0/build/ql/tmp/1967034913/10000
           Map Reduce
             Alias -> Map Operator Tree:
-              file:/data/users/athusoo/commits/hive_trunk_ws1/build/ql/tmp/430187082/10002 
+              file:/data/users/zshao/tools/699-trunk-apache-hive/.ptest_0/build/ql/tmp/1871117581/10002 
                   Reduce Output Operator
                     sort order: 
                     Map-reduce partition columns:
@@ -124,9 +124,9 @@
                           type: string
             Needs Tagging: false
             Path -> Alias:
-              file:/data/users/athusoo/commits/hive_trunk_ws1/build/ql/tmp/430187082/10002 
+              file:/data/users/zshao/tools/699-trunk-apache-hive/.ptest_0/build/ql/tmp/1871117581/10002 [file:/data/users/zshao/tools/699-trunk-apache-hive/.ptest_0/build/ql/tmp/1871117581/10002]
             Path -> Partition:
-              file:/data/users/athusoo/commits/hive_trunk_ws1/build/ql/tmp/430187082/10002 
+              file:/data/users/zshao/tools/699-trunk-apache-hive/.ptest_0/build/ql/tmp/1871117581/10002 
                 Partition
                 
                     input format: org.apache.hadoop.mapred.TextInputFormat
@@ -141,7 +141,7 @@
                       serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
                       file.inputformat org.apache.hadoop.mapred.TextInputFormat
                       file.outputformat org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-                      location file:/data/users/athusoo/commits/hive_trunk_ws1/build/ql/test/data/warehouse/dest1
+                      location file:/data/users/zshao/tools/699-trunk-apache-hive/.ptest_0/build/ql/test/data/warehouse/dest1
                     serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
                     name: dest1
             Reduce Operator Tree:
@@ -149,7 +149,7 @@
                 File Output Operator
                   compressed: false
                   GlobalTableId: 0
-                  directory: file:/data/users/athusoo/commits/hive_trunk_ws1/build/ql/tmp/1548094656/10000
+                  directory: file:/data/users/zshao/tools/699-trunk-apache-hive/.ptest_0/build/ql/tmp/1967034913/10000
                   table:
                       input format: org.apache.hadoop.mapred.TextInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
@@ -162,7 +162,7 @@
                         bucket_count -1
                         serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
                         file.inputformat org.apache.hadoop.mapred.TextInputFormat
-                        location file:/data/users/athusoo/commits/hive_trunk_ws1/build/ql/test/data/warehouse/dest1
+                        location file:/data/users/zshao/tools/699-trunk-apache-hive/.ptest_0/build/ql/test/data/warehouse/dest1
                         file.outputformat org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
                       serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
                       name: dest1
@@ -171,7 +171,7 @@
     Move Operator
       tables:
           replace: true
-          source: file:/data/users/athusoo/commits/hive_trunk_ws1/build/ql/tmp/1548094656/10000
+          source: file:/data/users/zshao/tools/699-trunk-apache-hive/.ptest_0/build/ql/tmp/1967034913/10000
           table:
               input format: org.apache.hadoop.mapred.TextInputFormat
               output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
@@ -185,10 +185,10 @@
                 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
                 file.inputformat org.apache.hadoop.mapred.TextInputFormat
                 file.outputformat org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-                location file:/data/users/athusoo/commits/hive_trunk_ws1/build/ql/test/data/warehouse/dest1
+                location file:/data/users/zshao/tools/699-trunk-apache-hive/.ptest_0/build/ql/test/data/warehouse/dest1
               serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
               name: dest1
-          tmp directory: file:/data/users/athusoo/commits/hive_trunk_ws1/build/ql/tmp/1548094656/10001
+          tmp directory: file:/data/users/zshao/tools/699-trunk-apache-hive/.ptest_0/build/ql/tmp/1967034913/10001
 
 
 query: FROM srcpart
@@ -197,7 +197,7 @@
 Output: default/dest1
 query: SELECT dest1.* FROM dest1
 Input: default/dest1
-Output: file:/data/users/athusoo/commits/hive_trunk_ws1/build/ql/tmp/1182712706/10000
+Output: file:/data/users/zshao/tools/699-trunk-apache-hive/.ptest_0/build/ql/tmp/75595335/10000
 86	val_86	12	2008-04-08
 27	val_27	12	2008-04-08
 98	val_98	12	2008-04-08