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/14 04:04:29 UTC

svn commit: r899038 - in /hadoop/hive/trunk: ./ ql/src/java/org/apache/hadoop/hive/ql/exec/ ql/src/java/org/apache/hadoop/hive/ql/udf/ ql/src/test/queries/clientpositive/ ql/src/test/results/clientpositive/ ql/src/test/results/compiler/plan/

Author: zshao
Date: Thu Jan 14 03:04:28 2010
New Revision: 899038

URL: http://svn.apache.org/viewvc?rev=899038&view=rev
Log:
HIVE-1045. (bigint % int) should return bigint instead of double. (Paul Yang via zshao)

Added:
    hadoop/hive/trunk/ql/src/test/queries/clientpositive/num_op_type_conv.q
    hadoop/hive/trunk/ql/src/test/results/clientpositive/num_op_type_conv.q.out
Modified:
    hadoop/hive/trunk/CHANGES.txt
    hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/ComparisonOpMethodResolver.java
    hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/NumericOpMethodResolver.java
    hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/UDFMethodResolver.java
    hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFOPDivide.java
    hadoop/hive/trunk/ql/src/test/results/clientpositive/input8.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/ppd_constant_expr.q.out
    hadoop/hive/trunk/ql/src/test/results/compiler/plan/input8.q.xml

Modified: hadoop/hive/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/CHANGES.txt?rev=899038&r1=899037&r2=899038&view=diff
==============================================================================
--- hadoop/hive/trunk/CHANGES.txt (original)
+++ hadoop/hive/trunk/CHANGES.txt Thu Jan 14 03:04:28 2010
@@ -19,8 +19,10 @@
     (Ning Zhang via namit)
 
     HIVE-1048. Make output of TestParse determinisitc
-    (Paul Yang Zhang via namit)
+    (Paul Yang via namit)
 
+    HIVE-1045. (bigint % int) should return bigint instead of double.
+    (Paul Yang via zshao)
 
 Release 0.5.0 -  Unreleased
 

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/ComparisonOpMethodResolver.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/ComparisonOpMethodResolver.java?rev=899038&r1=899037&r2=899038&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/ComparisonOpMethodResolver.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/ComparisonOpMethodResolver.java Thu Jan 14 03:04:28 2010
@@ -29,11 +29,13 @@
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils;
 
 /**
- * The class implements the method resolution for overloaded comparison operators. The
- * resolution logic is as follows:
- * 1. The resolver first tries to find an exact parameter match.
- * 2. If 1 fails and any of the parameters is a date, it converts the other to the date.
- * 3. If 1 and 3 fail then it returns the evaluate(Double, Double) method.
+ * The class implements the method resolution for operators like 
+ * (> < <= >= = <>). The resolution logic is as follows:
+ * 1. If one of the parameters is null, then it resolves to
+ *    evaluate(Double, Double)
+ * 2. If both of the parameters are of type T, then it resolves to 
+ *    evaluate(T, T)
+ * 3. If 1 and 2 fails then it resolves to evaluate(Double, Double).
  */
 public class ComparisonOpMethodResolver implements UDFMethodResolver {
 

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/NumericOpMethodResolver.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/NumericOpMethodResolver.java?rev=899038&r1=899037&r2=899038&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/NumericOpMethodResolver.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/NumericOpMethodResolver.java Thu Jan 14 03:04:28 2010
@@ -29,10 +29,17 @@
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils;
 
 /**
- * The class implements the method resolution for operators like (+, -, *, /, %, |, &, ^). The
- * resolution logic is as follows:
- * 1. The resolver first tries to find an exact parameter match.
- * 2. If 1 fails then it returns the evaluate(Double, Double) method.
+ * The class implements the method resolution for operators like 
+ * (+, -, *, %). The resolution logic is as follows:
+ * 
+ * 1. If one of the parameters is a string, then it resolves to
+ *    evaluate(double, double)
+ * 2. If one of the parameters is null, then it resolves to evaluate(T, T)
+ *    where T is the other non-null parameter type.
+ * 3. If both of the parameters are null, then it resolves to 
+ *    evaluate(byte, byte)
+ * 4. Otherwise, it resolves to evaluate(T, T), where T is the type resulting
+ *    from calling FunctionRegistry.getCommonClass() on the two arguments.
  */
 public class NumericOpMethodResolver implements UDFMethodResolver {
 
@@ -53,27 +60,44 @@
    */
   @Override
   public Method getEvalMethod(List<TypeInfo> argTypeInfos)
-  throws AmbiguousMethodException {
+  throws AmbiguousMethodException, UDFArgumentException {
     assert(argTypeInfos.size() == 2);
 
     List<TypeInfo> pTypeInfos = null;
-    if (argTypeInfos.get(0).equals(TypeInfoFactory.voidTypeInfo) ||
-        argTypeInfos.get(1).equals(TypeInfoFactory.voidTypeInfo)) {
-      pTypeInfos = new ArrayList<TypeInfo>();
-      pTypeInfos.add(TypeInfoFactory.doubleTypeInfo);
-      pTypeInfos.add(TypeInfoFactory.doubleTypeInfo);      
-    } else if (argTypeInfos.get(0).equals(TypeInfoFactory.stringTypeInfo) ||
+    List<TypeInfo> modArgTypeInfos = new ArrayList<TypeInfo>();
+
+    // If either argument is a string, we convert to a double because a number
+    // in string form should always be convertible into a double
+    if (argTypeInfos.get(0).equals(TypeInfoFactory.stringTypeInfo) ||
         argTypeInfos.get(1).equals(TypeInfoFactory.stringTypeInfo) ) {
-      pTypeInfos = new ArrayList<TypeInfo>();
-      pTypeInfos.add(TypeInfoFactory.doubleTypeInfo);
-      pTypeInfos.add(TypeInfoFactory.doubleTypeInfo);
-    } else if (argTypeInfos.get(0) == argTypeInfos.get(1)) {
-      pTypeInfos = argTypeInfos;
+      modArgTypeInfos.add(TypeInfoFactory.doubleTypeInfo);
+      modArgTypeInfos.add(TypeInfoFactory.doubleTypeInfo);
     } else {
-      pTypeInfos = new ArrayList<TypeInfo>();
-      pTypeInfos.add(TypeInfoFactory.doubleTypeInfo);
-      pTypeInfos.add(TypeInfoFactory.doubleTypeInfo);
+      // If it's a void, we change the type to a byte because once the types
+      // are run through getCommonClass(), a byte and any other type T will
+      // resolve to type T
+      for(int i=0; i<2; i++) {
+        if(argTypeInfos.get(i).equals(TypeInfoFactory.voidTypeInfo)) {
+          modArgTypeInfos.add(TypeInfoFactory.byteTypeInfo); 
+        } else {
+          modArgTypeInfos.add(argTypeInfos.get(i));
+        }
+      }
+    }
+    
+    TypeInfo commonType = FunctionRegistry.getCommonClass(
+        modArgTypeInfos.get(0), 
+        modArgTypeInfos.get(1));
+    
+    if(commonType == null) {
+      throw new UDFArgumentException("Unable to find a common class between" +
+      		"types " + modArgTypeInfos.get(0).getTypeName() + 
+      		" and " + modArgTypeInfos.get(1).getTypeName());
     }
+    
+    pTypeInfos = new ArrayList<TypeInfo>();
+    pTypeInfos.add(commonType);
+    pTypeInfos.add(commonType);
 
     Method udfMethod = null;
 

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/UDFMethodResolver.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/UDFMethodResolver.java?rev=899038&r1=899037&r2=899038&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/UDFMethodResolver.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/UDFMethodResolver.java Thu Jan 14 03:04:28 2010
@@ -43,5 +43,5 @@
    *                   function signature.
    */
   public Method getEvalMethod(List<TypeInfo> argClasses) 
-    throws AmbiguousMethodException;
+    throws AmbiguousMethodException, UDFArgumentException;
 }

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFOPDivide.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFOPDivide.java?rev=899038&r1=899037&r2=899038&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFOPDivide.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFOPDivide.java Thu Jan 14 03:04:28 2010
@@ -37,12 +37,17 @@
         "  > SELECT 3 _FUNC_ 2 FROM src LIMIT 1;\n" +
         "  1.5"
 )
+/**
+ * Note that in SQL, the return type of divide is not necessarily the same 
+ * as the parameters. For example, 3 / 2 = 1.5, not 1. To follow SQL, we always
+ * return a double for divide.
+ */
 public class UDFOPDivide extends UDF {
 
   private static Log LOG = LogFactory.getLog("org.apache.hadoop.hive.ql.udf.UDFOPDivide");
 
   protected DoubleWritable doubleWritable = new DoubleWritable();
-  
+ 
   public DoubleWritable evaluate(DoubleWritable a, DoubleWritable b)  {
     // LOG.info("Get input " + a.getClass() + ":" + a + " " + b.getClass() + ":" + b);
     if ((a == null) || (b == null))

Added: hadoop/hive/trunk/ql/src/test/queries/clientpositive/num_op_type_conv.q
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/queries/clientpositive/num_op_type_conv.q?rev=899038&view=auto
==============================================================================
--- hadoop/hive/trunk/ql/src/test/queries/clientpositive/num_op_type_conv.q (added)
+++ hadoop/hive/trunk/ql/src/test/queries/clientpositive/num_op_type_conv.q Thu Jan 14 03:04:28 2010
@@ -0,0 +1,9 @@
+EXPLAIN SELECT null + 7, 1.0 - null, null + null,
+               CAST(21 AS BIGINT) % CAST(5 AS TINYINT),
+               CAST(21 AS BIGINT) % CAST(21 AS BIGINT),
+               9 % "3" FROM src LIMIT 1;
+
+SELECT null + 7, 1.0 - null, null + null,
+       CAST(21 AS BIGINT) % CAST(5 AS TINYINT),
+       CAST(21 AS BIGINT) % CAST(21 AS BIGINT),
+       9 % "3" FROM src LIMIT 1;

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=899038&r1=899037&r2=899038&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 Thu Jan 14 03:04:28 2010
@@ -29,19 +29,19 @@
             Select Operator
               expressions:
                     expr: (4 + null)
-                    type: double
+                    type: int
                     expr: (key - null)
                     type: double
                     expr: (null + null)
-                    type: double
+                    type: tinyint
               outputColumnNames: _col0, _col1, _col2
               Select Operator
                 expressions:
                       expr: _col0
-                      type: double
+                      type: int
                       expr: UDFToInteger(_col1)
                       type: int
-                      expr: _col2
+                      expr: UDFToDouble(_col2)
                       type: double
                 outputColumnNames: _col0, _col1, _col2
                 File Output Operator
@@ -59,10 +59,10 @@
           Move Operator
             files:
                 hdfs directory: true
-                destination: file:/data/users/njain/hive5/hive5/build/ql/tmp/539046228/10000
+                destination: file:/data/users/pyang/task/trunk/VENDOR.hive/trunk/build/ql/tmp/1595941981/10000
           Map Reduce
             Alias -> Map Operator Tree:
-              file:/data/users/njain/hive5/hive5/build/ql/tmp/1447024085/10002 
+              file:/data/users/pyang/task/trunk/VENDOR.hive/trunk/build/ql/tmp/1940258641/10002 
                   Reduce Output Operator
                     sort order: 
                     Map-reduce partition columns:
@@ -111,11 +111,11 @@
 PREHOOK: query: SELECT dest1.* FROM dest1
 PREHOOK: type: QUERY
 PREHOOK: Input: default@dest1
-PREHOOK: Output: file:/data/users/njain/hive5/hive5/build/ql/tmp/1333594748/10000
+PREHOOK: Output: file:/data/users/pyang/task/trunk/VENDOR.hive/trunk/build/ql/tmp/398951059/10000
 POSTHOOK: query: SELECT dest1.* FROM dest1
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@dest1
-POSTHOOK: Output: file:/data/users/njain/hive5/hive5/build/ql/tmp/1333594748/10000
+POSTHOOK: Output: file:/data/users/pyang/task/trunk/VENDOR.hive/trunk/build/ql/tmp/398951059/10000
 NULL	NULL	NULL
 NULL	NULL	NULL
 NULL	NULL	NULL

Added: hadoop/hive/trunk/ql/src/test/results/clientpositive/num_op_type_conv.q.out
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/results/clientpositive/num_op_type_conv.q.out?rev=899038&view=auto
==============================================================================
--- hadoop/hive/trunk/ql/src/test/results/clientpositive/num_op_type_conv.q.out (added)
+++ hadoop/hive/trunk/ql/src/test/results/clientpositive/num_op_type_conv.q.out Thu Jan 14 03:04:28 2010
@@ -0,0 +1,67 @@
+PREHOOK: query: EXPLAIN SELECT null + 7, 1.0 - null, null + null,
+               CAST(21 AS BIGINT) % CAST(5 AS TINYINT),
+               CAST(21 AS BIGINT) % CAST(21 AS BIGINT),
+               9 % "3" FROM src LIMIT 1
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN SELECT null + 7, 1.0 - null, null + null,
+               CAST(21 AS BIGINT) % CAST(5 AS TINYINT),
+               CAST(21 AS BIGINT) % CAST(21 AS BIGINT),
+               9 % "3" FROM src LIMIT 1
+POSTHOOK: type: QUERY
+ABSTRACT SYNTAX TREE:
+  (TOK_QUERY (TOK_FROM (TOK_TABREF src)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (+ TOK_NULL 7)) (TOK_SELEXPR (- 1.0 TOK_NULL)) (TOK_SELEXPR (+ TOK_NULL TOK_NULL)) (TOK_SELEXPR (% (TOK_FUNCTION TOK_BIGINT 21) (TOK_FUNCTION TOK_TINYINT 5))) (TOK_SELEXPR (% (TOK_FUNCTION TOK_BIGINT 21) (TOK_FUNCTION TOK_BIGINT 21))) (TOK_SELEXPR (% 9 "3"))) (TOK_LIMIT 1)))
+
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 is a root stage
+
+STAGE PLANS:
+  Stage: Stage-1
+    Map Reduce
+      Alias -> Map Operator Tree:
+        src 
+          TableScan
+            alias: src
+            Select Operator
+              expressions:
+                    expr: (null + 7)
+                    type: int
+                    expr: (1.0 - null)
+                    type: double
+                    expr: (null + null)
+                    type: tinyint
+                    expr: (UDFToLong(21) % UDFToByte(5))
+                    type: bigint
+                    expr: (UDFToLong(21) % UDFToLong(21))
+                    type: bigint
+                    expr: (9 % '3')
+                    type: double
+              outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
+              Limit
+                File Output Operator
+                  compressed: false
+                  GlobalTableId: 0
+                  table:
+                      input format: org.apache.hadoop.mapred.TextInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: 1
+
+
+PREHOOK: query: SELECT null + 7, 1.0 - null, null + null,
+       CAST(21 AS BIGINT) % CAST(5 AS TINYINT),
+       CAST(21 AS BIGINT) % CAST(21 AS BIGINT),
+       9 % "3" FROM src LIMIT 1
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+PREHOOK: Output: file:/data/users/pyang/merge/trunk/VENDOR.hive/trunk/build/ql/tmp/1374914728/10000
+POSTHOOK: query: SELECT null + 7, 1.0 - null, null + null,
+       CAST(21 AS BIGINT) % CAST(5 AS TINYINT),
+       CAST(21 AS BIGINT) % CAST(21 AS BIGINT),
+       9 % "3" FROM src LIMIT 1
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+POSTHOOK: Output: file:/data/users/pyang/merge/trunk/VENDOR.hive/trunk/build/ql/tmp/1374914728/10000
+NULL	NULL	NULL	1	0	0.0

Modified: hadoop/hive/trunk/ql/src/test/results/clientpositive/ppd_constant_expr.q.out
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/results/clientpositive/ppd_constant_expr.q.out?rev=899038&r1=899037&r2=899038&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/test/results/clientpositive/ppd_constant_expr.q.out (original)
+++ hadoop/hive/trunk/ql/src/test/results/clientpositive/ppd_constant_expr.q.out Thu Jan 14 03:04:28 2010
@@ -29,19 +29,19 @@
             Select Operator
               expressions:
                     expr: (4 + null)
-                    type: double
+                    type: int
                     expr: (key - null)
                     type: double
                     expr: (null + null)
-                    type: double
+                    type: tinyint
               outputColumnNames: _col0, _col1, _col2
               Select Operator
                 expressions:
                       expr: _col0
-                      type: double
+                      type: int
                       expr: UDFToInteger(_col1)
                       type: int
-                      expr: _col2
+                      expr: UDFToDouble(_col2)
                       type: double
                 outputColumnNames: _col0, _col1, _col2
                 File Output Operator
@@ -59,10 +59,10 @@
           Move Operator
             files:
                 hdfs directory: true
-                destination: file:/data/users/njain/hive5/hive5/build/ql/tmp/1422233089/10000
+                destination: file:/data/users/pyang/task/trunk/VENDOR.hive/trunk/build/ql/tmp/1689506606/10000
           Map Reduce
             Alias -> Map Operator Tree:
-              file:/data/users/njain/hive5/hive5/build/ql/tmp/70851087/10002 
+              file:/data/users/pyang/task/trunk/VENDOR.hive/trunk/build/ql/tmp/552481309/10002 
                   Reduce Output Operator
                     sort order: 
                     Map-reduce partition columns:
@@ -111,11 +111,11 @@
 PREHOOK: query: SELECT ppd_constant_expr.* FROM ppd_constant_expr
 PREHOOK: type: QUERY
 PREHOOK: Input: default@ppd_constant_expr
-PREHOOK: Output: file:/data/users/njain/hive5/hive5/build/ql/tmp/1300457424/10000
+PREHOOK: Output: file:/data/users/pyang/task/trunk/VENDOR.hive/trunk/build/ql/tmp/1743239897/10000
 POSTHOOK: query: SELECT ppd_constant_expr.* FROM ppd_constant_expr
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@ppd_constant_expr
-POSTHOOK: Output: file:/data/users/njain/hive5/hive5/build/ql/tmp/1300457424/10000
+POSTHOOK: Output: file:/data/users/pyang/task/trunk/VENDOR.hive/trunk/build/ql/tmp/1743239897/10000
 NULL	NULL	NULL
 NULL	NULL	NULL
 NULL	NULL	NULL

Modified: hadoop/hive/trunk/ql/src/test/results/compiler/plan/input8.q.xml
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/results/compiler/plan/input8.q.xml?rev=899038&r1=899037&r2=899038&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/test/results/compiler/plan/input8.q.xml (original)
+++ hadoop/hive/trunk/ql/src/test/results/compiler/plan/input8.q.xml Thu Jan 14 03:04:28 2010
@@ -62,11 +62,11 @@
          </void> 
          <void method="put"> 
           <string>location</string> 
-          <string>file:/data/users/pyang/mstore/trunk/VENDOR.hive/trunk/build/ql/test/data/warehouse/src1</string> 
+          <string>file:/data/users/pyang/merge/trunk/VENDOR.hive/trunk/build/ql/test/data/warehouse/src1</string> 
          </void> 
          <void method="put"> 
           <string>transient_lastDdlTime</string> 
-          <string>1263415297</string> 
+          <string>1263432182</string> 
          </void> 
         </object> 
        </void> 
@@ -111,7 +111,7 @@
                 <void property="conf"> 
                  <object class="org.apache.hadoop.hive.ql.plan.fileSinkDesc"> 
                   <void property="dirName"> 
-                   <string>file:/data/users/pyang/mstore/trunk/VENDOR.hive/trunk/build/ql/tmp/37685710/10001</string> 
+                   <string>file:/data/users/pyang/merge/trunk/VENDOR.hive/trunk/build/ql/tmp/1534510969/10001</string> 
                   </void> 
                   <void property="tableInfo"> 
                    <object class="org.apache.hadoop.hive.ql.plan.tableDesc"> 
@@ -136,7 +136,7 @@
                       </void> 
                       <void method="put"> 
                        <string>columns.types</string> 
-                       <string>double:double:double</string> 
+                       <string>int:double:tinyint</string> 
                       </void> 
                      </object> 
                     </void> 
@@ -182,7 +182,7 @@
                       <void property="type"> 
                        <object id="PrimitiveTypeInfo0" class="org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo"> 
                         <void property="typeName"> 
-                         <string>double</string> 
+                         <string>int</string> 
                         </void> 
                        </object> 
                       </void> 
@@ -194,7 +194,11 @@
                        <string>_col1</string> 
                       </void> 
                       <void property="type"> 
-                       <object idref="PrimitiveTypeInfo0"/> 
+                       <object id="PrimitiveTypeInfo1" class="org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo"> 
+                        <void property="typeName"> 
+                         <string>double</string> 
+                        </void> 
+                       </object> 
                       </void> 
                      </object> 
                     </void> 
@@ -204,7 +208,11 @@
                        <string>_col2</string> 
                       </void> 
                       <void property="type"> 
-                       <object idref="PrimitiveTypeInfo0"/> 
+                       <object id="PrimitiveTypeInfo2" class="org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo"> 
+                        <void property="typeName"> 
+                         <string>tinyint</string> 
+                        </void> 
+                       </object> 
                       </void> 
                      </object> 
                     </void> 
@@ -245,7 +253,7 @@
                  </object> 
                 </void> 
                 <void property="typeInfo"> 
-                 <object idref="PrimitiveTypeInfo0"/> 
+                 <object idref="PrimitiveTypeInfo2"/> 
                 </void> 
                </object> 
               </void> 
@@ -263,7 +271,7 @@
                      <string>src1</string> 
                     </void> 
                     <void property="typeInfo"> 
-                     <object id="PrimitiveTypeInfo1" class="org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo"> 
+                     <object id="PrimitiveTypeInfo3" class="org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo"> 
                       <void property="typeName"> 
                        <string>string</string> 
                       </void> 
@@ -290,7 +298,7 @@
                  </object> 
                 </void> 
                 <void property="typeInfo"> 
-                 <object idref="PrimitiveTypeInfo0"/> 
+                 <object idref="PrimitiveTypeInfo1"/> 
                 </void> 
                </object> 
               </void> 
@@ -302,11 +310,7 @@
                   <void method="add"> 
                    <object class="org.apache.hadoop.hive.ql.plan.exprNodeConstantDesc"> 
                     <void property="typeInfo"> 
-                     <object class="org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo"> 
-                      <void property="typeName"> 
-                       <string>int</string> 
-                      </void> 
-                     </object> 
+                     <object idref="PrimitiveTypeInfo0"/> 
                     </void> 
                     <void property="value"> 
                      <int>4</int> 
@@ -420,7 +424,7 @@
                    <string>_col1</string> 
                   </void> 
                   <void property="type"> 
-                   <object idref="PrimitiveTypeInfo0"/> 
+                   <object idref="PrimitiveTypeInfo1"/> 
                   </void> 
                  </object> 
                 </void> 
@@ -433,7 +437,7 @@
                    <string>_col2</string> 
                   </void> 
                   <void property="type"> 
-                   <object idref="PrimitiveTypeInfo0"/> 
+                   <object idref="PrimitiveTypeInfo2"/> 
                   </void> 
                  </object> 
                 </void> 
@@ -488,7 +492,7 @@
                <string>key</string> 
               </void> 
               <void property="type"> 
-               <object idref="PrimitiveTypeInfo1"/> 
+               <object idref="PrimitiveTypeInfo3"/> 
               </void> 
              </object> 
             </void> 
@@ -498,7 +502,7 @@
                <string>value</string> 
               </void> 
               <void property="type"> 
-               <object idref="PrimitiveTypeInfo1"/> 
+               <object idref="PrimitiveTypeInfo3"/> 
               </void> 
              </object> 
             </void> 
@@ -513,7 +517,7 @@
     <void property="pathToAliases"> 
      <object class="java.util.LinkedHashMap"> 
       <void method="put"> 
-       <string>file:/data/users/pyang/mstore/trunk/VENDOR.hive/trunk/build/ql/test/data/warehouse/src1</string> 
+       <string>file:/data/users/pyang/merge/trunk/VENDOR.hive/trunk/build/ql/test/data/warehouse/src1</string> 
        <object class="java.util.ArrayList"> 
         <void method="add"> 
          <string>src1</string> 
@@ -525,7 +529,7 @@
     <void property="pathToPartitionInfo"> 
      <object class="java.util.LinkedHashMap"> 
       <void method="put"> 
-       <string>file:/data/users/pyang/mstore/trunk/VENDOR.hive/trunk/build/ql/test/data/warehouse/src1</string> 
+       <string>file:/data/users/pyang/merge/trunk/VENDOR.hive/trunk/build/ql/test/data/warehouse/src1</string> 
        <object class="org.apache.hadoop.hive.ql.plan.partitionDesc"> 
         <void property="deserializerClass"> 
          <class>org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe</class>