You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by rh...@apache.org on 2014/03/31 23:04:09 UTC

svn commit: r1583463 [1/2] - in /hive/branches/branch-0.13: metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ metastore/src/java/org/apache/hadoop/hive/metastore/ ql/src/java/org/apache/hadoop/hive/ql/exec/ ql/src/java/org/apa...

Author: rhbutani
Date: Mon Mar 31 21:04:07 2014
New Revision: 1583463

URL: http://svn.apache.org/r1583463
Log:
HIVE-6642 Query fails to vectorize when a non string partition column is part of the query expression (Hari Subramaniyan via Harish Butani)

Modified:
    hive/branches/branch-0.13/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/hive_metastoreConstants.java
    hive/branches/branch-0.13/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreUtils.java
    hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/exec/FetchOperator.java
    hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/exec/MapOperator.java
    hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorGroupByOperator.java
    hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizedRowBatchCtx.java
    hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
    hive/branches/branch-0.13/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorizedRowBatchCtx.java
    hive/branches/branch-0.13/ql/src/test/queries/clientpositive/alter_partition_coltype.q
    hive/branches/branch-0.13/ql/src/test/queries/clientpositive/pcr.q
    hive/branches/branch-0.13/ql/src/test/results/clientpositive/alter_partition_coltype.q.out
    hive/branches/branch-0.13/ql/src/test/results/clientpositive/annotate_stats_part.q.out
    hive/branches/branch-0.13/ql/src/test/results/clientpositive/auto_sortmerge_join_1.q.out
    hive/branches/branch-0.13/ql/src/test/results/clientpositive/auto_sortmerge_join_11.q.out
    hive/branches/branch-0.13/ql/src/test/results/clientpositive/auto_sortmerge_join_12.q.out
    hive/branches/branch-0.13/ql/src/test/results/clientpositive/auto_sortmerge_join_2.q.out
    hive/branches/branch-0.13/ql/src/test/results/clientpositive/auto_sortmerge_join_3.q.out
    hive/branches/branch-0.13/ql/src/test/results/clientpositive/auto_sortmerge_join_4.q.out
    hive/branches/branch-0.13/ql/src/test/results/clientpositive/auto_sortmerge_join_7.q.out
    hive/branches/branch-0.13/ql/src/test/results/clientpositive/auto_sortmerge_join_8.q.out
    hive/branches/branch-0.13/ql/src/test/results/clientpositive/bucket3.q.out
    hive/branches/branch-0.13/ql/src/test/results/clientpositive/bucketcontext_1.q.out
    hive/branches/branch-0.13/ql/src/test/results/clientpositive/bucketcontext_2.q.out
    hive/branches/branch-0.13/ql/src/test/results/clientpositive/bucketcontext_3.q.out
    hive/branches/branch-0.13/ql/src/test/results/clientpositive/bucketcontext_4.q.out
    hive/branches/branch-0.13/ql/src/test/results/clientpositive/bucketcontext_6.q.out
    hive/branches/branch-0.13/ql/src/test/results/clientpositive/bucketcontext_7.q.out
    hive/branches/branch-0.13/ql/src/test/results/clientpositive/bucketcontext_8.q.out
    hive/branches/branch-0.13/ql/src/test/results/clientpositive/bucketmapjoin1.q.out
    hive/branches/branch-0.13/ql/src/test/results/clientpositive/bucketmapjoin10.q.out
    hive/branches/branch-0.13/ql/src/test/results/clientpositive/bucketmapjoin11.q.out
    hive/branches/branch-0.13/ql/src/test/results/clientpositive/bucketmapjoin12.q.out
    hive/branches/branch-0.13/ql/src/test/results/clientpositive/bucketmapjoin13.q.out
    hive/branches/branch-0.13/ql/src/test/results/clientpositive/bucketmapjoin2.q.out
    hive/branches/branch-0.13/ql/src/test/results/clientpositive/bucketmapjoin3.q.out
    hive/branches/branch-0.13/ql/src/test/results/clientpositive/bucketmapjoin5.q.out
    hive/branches/branch-0.13/ql/src/test/results/clientpositive/bucketmapjoin7.q.out
    hive/branches/branch-0.13/ql/src/test/results/clientpositive/bucketmapjoin8.q.out
    hive/branches/branch-0.13/ql/src/test/results/clientpositive/bucketmapjoin9.q.out
    hive/branches/branch-0.13/ql/src/test/results/clientpositive/bucketmapjoin_negative.q.out
    hive/branches/branch-0.13/ql/src/test/results/clientpositive/bucketmapjoin_negative2.q.out
    hive/branches/branch-0.13/ql/src/test/results/clientpositive/columnstats_partlvl.q.out
    hive/branches/branch-0.13/ql/src/test/results/clientpositive/combine2_hadoop20.q.out
    hive/branches/branch-0.13/ql/src/test/results/clientpositive/dynamic_partition_skip_default.q.out
    hive/branches/branch-0.13/ql/src/test/results/clientpositive/filter_join_breaktask.q.out
    hive/branches/branch-0.13/ql/src/test/results/clientpositive/filter_numeric.q.out
    hive/branches/branch-0.13/ql/src/test/results/clientpositive/groupby_map_ppr.q.out
    hive/branches/branch-0.13/ql/src/test/results/clientpositive/groupby_map_ppr_multi_distinct.q.out
    hive/branches/branch-0.13/ql/src/test/results/clientpositive/groupby_ppr.q.out
    hive/branches/branch-0.13/ql/src/test/results/clientpositive/groupby_ppr_multi_distinct.q.out
    hive/branches/branch-0.13/ql/src/test/results/clientpositive/groupby_sort_6.q.out
    hive/branches/branch-0.13/ql/src/test/results/clientpositive/input23.q.out
    hive/branches/branch-0.13/ql/src/test/results/clientpositive/input42.q.out
    hive/branches/branch-0.13/ql/src/test/results/clientpositive/input_part1.q.out
    hive/branches/branch-0.13/ql/src/test/results/clientpositive/input_part2.q.out
    hive/branches/branch-0.13/ql/src/test/results/clientpositive/input_part7.q.out
    hive/branches/branch-0.13/ql/src/test/results/clientpositive/input_part9.q.out
    hive/branches/branch-0.13/ql/src/test/results/clientpositive/join26.q.out
    hive/branches/branch-0.13/ql/src/test/results/clientpositive/join32.q.out
    hive/branches/branch-0.13/ql/src/test/results/clientpositive/join32_lessSize.q.out
    hive/branches/branch-0.13/ql/src/test/results/clientpositive/join33.q.out
    hive/branches/branch-0.13/ql/src/test/results/clientpositive/join9.q.out
    hive/branches/branch-0.13/ql/src/test/results/clientpositive/join_map_ppr.q.out
    hive/branches/branch-0.13/ql/src/test/results/clientpositive/list_bucket_dml_10.q.out
    hive/branches/branch-0.13/ql/src/test/results/clientpositive/load_dyn_part8.q.out
    hive/branches/branch-0.13/ql/src/test/results/clientpositive/louter_join_ppr.q.out
    hive/branches/branch-0.13/ql/src/test/results/clientpositive/merge3.q.out
    hive/branches/branch-0.13/ql/src/test/results/clientpositive/metadataonly1.q.out
    hive/branches/branch-0.13/ql/src/test/results/clientpositive/outer_join_ppr.q.out
    hive/branches/branch-0.13/ql/src/test/results/clientpositive/pcr.q.out
    hive/branches/branch-0.13/ql/src/test/results/clientpositive/ppd_union_view.q.out
    hive/branches/branch-0.13/ql/src/test/results/clientpositive/ppd_vc.q.out
    hive/branches/branch-0.13/ql/src/test/results/clientpositive/ppr_allchildsarenull.q.out
    hive/branches/branch-0.13/ql/src/test/results/clientpositive/push_or.q.out
    hive/branches/branch-0.13/ql/src/test/results/clientpositive/rand_partitionpruner2.q.out
    hive/branches/branch-0.13/ql/src/test/results/clientpositive/rand_partitionpruner3.q.out
    hive/branches/branch-0.13/ql/src/test/results/clientpositive/reduce_deduplicate.q.out
    hive/branches/branch-0.13/ql/src/test/results/clientpositive/router_join_ppr.q.out
    hive/branches/branch-0.13/ql/src/test/results/clientpositive/sample1.q.out
    hive/branches/branch-0.13/ql/src/test/results/clientpositive/sample10.q.out
    hive/branches/branch-0.13/ql/src/test/results/clientpositive/sample8.q.out
    hive/branches/branch-0.13/ql/src/test/results/clientpositive/smb_mapjoin_11.q.out
    hive/branches/branch-0.13/ql/src/test/results/clientpositive/smb_mapjoin_12.q.out
    hive/branches/branch-0.13/ql/src/test/results/clientpositive/sort_merge_join_desc_5.q.out
    hive/branches/branch-0.13/ql/src/test/results/clientpositive/sort_merge_join_desc_6.q.out
    hive/branches/branch-0.13/ql/src/test/results/clientpositive/sort_merge_join_desc_7.q.out
    hive/branches/branch-0.13/ql/src/test/results/clientpositive/stats11.q.out
    hive/branches/branch-0.13/ql/src/test/results/clientpositive/stats12.q.out
    hive/branches/branch-0.13/ql/src/test/results/clientpositive/stats13.q.out
    hive/branches/branch-0.13/ql/src/test/results/clientpositive/transform_ppr1.q.out
    hive/branches/branch-0.13/ql/src/test/results/clientpositive/transform_ppr2.q.out
    hive/branches/branch-0.13/ql/src/test/results/clientpositive/union22.q.out
    hive/branches/branch-0.13/ql/src/test/results/clientpositive/union_ppr.q.out
    hive/branches/branch-0.13/ql/src/test/results/compiler/plan/input2.q.xml
    hive/branches/branch-0.13/ql/src/test/results/compiler/plan/input3.q.xml
    hive/branches/branch-0.13/ql/src/test/results/compiler/plan/input_part1.q.xml
    hive/branches/branch-0.13/ql/src/test/results/compiler/plan/sample1.q.xml
    hive/branches/branch-0.13/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaHiveVarcharObjectInspector.java

Modified: hive/branches/branch-0.13/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/hive_metastoreConstants.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/hive_metastoreConstants.java?rev=1583463&r1=1583462&r2=1583463&view=diff
==============================================================================
--- hive/branches/branch-0.13/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/hive_metastoreConstants.java (original)
+++ hive/branches/branch-0.13/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/hive_metastoreConstants.java Mon Mar 31 21:04:07 2014
@@ -66,6 +66,8 @@ public class hive_metastoreConstants {
   public static final String META_TABLE_SERDE = "serde";
 
   public static final String META_TABLE_PARTITION_COLUMNS = "partition_columns";
+  
+  public static final String META_TABLE_PARTITION_COLUMN_TYPES = "partition_columns.types";
 
   public static final String FILE_INPUT_FORMAT = "file.inputformat";
 

Modified: hive/branches/branch-0.13/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreUtils.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreUtils.java?rev=1583463&r1=1583462&r2=1583463&view=diff
==============================================================================
--- hive/branches/branch-0.13/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreUtils.java (original)
+++ hive/branches/branch-0.13/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreUtils.java Mon Mar 31 21:04:07 2014
@@ -957,11 +957,16 @@ public class MetaStoreUtils {
 
     String partString = "";
     String partStringSep = "";
+    String partTypesString = "";
+    String partTypesStringSep = "";
     for (FieldSchema partKey : partitionKeys) {
       partString = partString.concat(partStringSep);
       partString = partString.concat(partKey.getName());
+      partTypesString = partTypesString.concat(partTypesStringSep);
+      partTypesString = partTypesString.concat(partKey.getType());      
       if (partStringSep.length() == 0) {
         partStringSep = "/";
+        partTypesStringSep = ":";
       }
     }
     if (partString.length() > 0) {
@@ -969,6 +974,10 @@ public class MetaStoreUtils {
           .setProperty(
               org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.META_TABLE_PARTITION_COLUMNS,
               partString);
+      schema
+      .setProperty(
+          org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.META_TABLE_PARTITION_COLUMN_TYPES,
+          partTypesString);      
     }
 
     if (parameters != null) {

Modified: hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/exec/FetchOperator.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/exec/FetchOperator.java?rev=1583463&r1=1583462&r2=1583463&view=diff
==============================================================================
--- hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/exec/FetchOperator.java (original)
+++ hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/exec/FetchOperator.java Mon Mar 31 21:04:07 2014
@@ -59,6 +59,11 @@ import org.apache.hadoop.hive.serde2.obj
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory;
 import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector;
 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.PrimitiveObjectInspectorUtils.PrimitiveTypeEntry;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils;
+import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableComparable;
 import org.apache.hadoop.mapred.InputFormat;
@@ -245,9 +250,12 @@ public class FetchOperator implements Se
     String pcols = partition.getTableDesc().getProperties().getProperty(
         org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.META_TABLE_PARTITION_COLUMNS);
     String[] partKeys = pcols.trim().split("/");
-    row[1] = createPartValue(partKeys, partition.getPartSpec());
+    String pcolTypes = partition.getTableDesc().getProperties().getProperty(
+        org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.META_TABLE_PARTITION_COLUMN_TYPES); 
+    String[] partKeyTypes = pcolTypes.trim().split(":");
+    row[1] = createPartValue(partKeys, partition.getPartSpec(), partKeyTypes);
 
-    return createRowInspector(getStructOIFrom(partitionOI), partKeys);
+    return createRowInspector(getStructOIFrom(partitionOI), partKeys, partKeyTypes);
   }
 
   private StructObjectInspector getRowInspectorFromPartitionedTable(TableDesc table)
@@ -257,8 +265,11 @@ public class FetchOperator implements Se
     String pcols = table.getProperties().getProperty(
         org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.META_TABLE_PARTITION_COLUMNS);
     String[] partKeys = pcols.trim().split("/");
+    String pcolTypes = table.getProperties().getProperty(
+        org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.META_TABLE_PARTITION_COLUMN_TYPES); 
+    String[] partKeyTypes = pcolTypes.trim().split(":");    
     row[1] = null;
-    return createRowInspector(getStructOIFrom(serde.getObjectInspector()), partKeys);
+    return createRowInspector(getStructOIFrom(serde.getObjectInspector()), partKeys, partKeyTypes);
   }
 
   private StructObjectInspector getStructOIFrom(ObjectInspector current) throws SerDeException {
@@ -276,13 +287,16 @@ public class FetchOperator implements Se
         Arrays.asList(current, vcsOI)) : current;
   }
 
-  private StructObjectInspector createRowInspector(StructObjectInspector current, String[] partKeys)
+  private StructObjectInspector createRowInspector(StructObjectInspector current, String[] partKeys, String[] partKeyTypes)
       throws SerDeException {
     List<String> partNames = new ArrayList<String>();
     List<ObjectInspector> partObjectInspectors = new ArrayList<ObjectInspector>();
-    for (String key : partKeys) {
-      partNames.add(key);
-      partObjectInspectors.add(PrimitiveObjectInspectorFactory.javaStringObjectInspector);
+    for (int i = 0; i < partKeys.length; i++) {
+      String key = partKeys[i];
+      partNames.add(key);    
+      ObjectInspector oi = TypeInfoUtils.getStandardJavaObjectInspectorFromTypeInfo(
+          TypeInfoFactory.getPrimitiveTypeInfo(partKeyTypes[i]));
+      partObjectInspectors.add(oi);
     }
     StructObjectInspector partObjectInspector = ObjectInspectorFactory
         .getStandardStructObjectInspector(partNames, partObjectInspectors);
@@ -292,10 +306,16 @@ public class FetchOperator implements Se
             Arrays.asList(current, partObjectInspector));
   }
 
-  private List<String> createPartValue(String[] partKeys, Map<String, String> partSpec) {
-    List<String> partValues = new ArrayList<String>();
-    for (String key : partKeys) {
-      partValues.add(partSpec.get(key));
+  private Object[] createPartValue(String[] partKeys, Map<String, String> partSpec, String[] partKeyTypes) {
+    Object[] partValues = new Object[partKeys.length];
+    for (int i = 0; i < partKeys.length; i++) {
+      String key = partKeys[i];
+      ObjectInspector oi = TypeInfoUtils.getStandardJavaObjectInspectorFromTypeInfo(
+          TypeInfoFactory.getPrimitiveTypeInfo(partKeyTypes[i]));
+      partValues[i] = 
+          ObjectInspectorConverters.
+          getConverter(PrimitiveObjectInspectorFactory.
+              javaStringObjectInspector, oi).convert(partSpec.get(key));   
     }
     return partValues;
   }

Modified: hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/exec/MapOperator.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/exec/MapOperator.java?rev=1583463&r1=1583462&r2=1583463&view=diff
==============================================================================
--- hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/exec/MapOperator.java (original)
+++ hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/exec/MapOperator.java Mon Mar 31 21:04:07 2014
@@ -53,6 +53,7 @@ import org.apache.hadoop.hive.serde2.obj
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory;
 import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Writable;
@@ -179,7 +180,7 @@ public class MapOperator extends Operato
 
     PartitionDesc pd = ctx.partDesc;
     TableDesc td = pd.getTableDesc();
-
+    
     MapOpCtx opCtx = new MapOpCtx();
     // Use table properties in case of unpartitioned tables,
     // and the union of table properties and partition properties, with partition
@@ -203,28 +204,43 @@ public class MapOperator extends Operato
 
     opCtx.partTblObjectInspectorConverter = ObjectInspectorConverters.getConverter(
         partRawRowObjectInspector, opCtx.tblRawRowObjectInspector);
-
+    
     // Next check if this table has partitions and if so
     // get the list of partition names as well as allocate
     // the serdes for the partition columns
     String pcols = partProps.getProperty(hive_metastoreConstants.META_TABLE_PARTITION_COLUMNS);
-    // Log LOG = LogFactory.getLog(MapOperator.class.getName());
+    
     if (pcols != null && pcols.length() > 0) {
       String[] partKeys = pcols.trim().split("/");
+      String pcolTypes = partProps.getProperty(hive_metastoreConstants.META_TABLE_PARTITION_COLUMN_TYPES);      
+      String[] partKeyTypes = pcolTypes.trim().split(":");
+      
+      if (partKeys.length > partKeyTypes.length) {
+          throw new HiveException("Internal error : partKeys length, " +partKeys.length +
+                  " greater than partKeyTypes length, " + partKeyTypes.length);
+      }
+      
       List<String> partNames = new ArrayList<String>(partKeys.length);
       Object[] partValues = new Object[partKeys.length];
       List<ObjectInspector> partObjectInspectors = new ArrayList<ObjectInspector>(partKeys.length);
+      
       for (int i = 0; i < partKeys.length; i++) {
         String key = partKeys[i];
         partNames.add(key);
+        ObjectInspector oi = PrimitiveObjectInspectorFactory.getPrimitiveWritableObjectInspector
+            (TypeInfoFactory.getPrimitiveTypeInfo(partKeyTypes[i]));
+        
         // Partitions do not exist for this table
         if (partSpec == null) {
           // for partitionless table, initialize partValue to null
           partValues[i] = null;
         } else {
-          partValues[i] = new Text(partSpec.get(key));
+            partValues[i] = 
+                ObjectInspectorConverters.
+                getConverter(PrimitiveObjectInspectorFactory.
+                    javaStringObjectInspector, oi).convert(partSpec.get(key)); 
         }
-        partObjectInspectors.add(PrimitiveObjectInspectorFactory.writableStringObjectInspector);
+        partObjectInspectors.add(oi);
       }
       opCtx.rowWithPart = new Object[] {null, partValues};
       opCtx.partObjectInspector = ObjectInspectorFactory

Modified: hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorGroupByOperator.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorGroupByOperator.java?rev=1583463&r1=1583462&r2=1583463&view=diff
==============================================================================
--- hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorGroupByOperator.java (original)
+++ hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorGroupByOperator.java Mon Mar 31 21:04:07 2014
@@ -260,7 +260,7 @@ public class VectorGroupByOperator exten
             HiveConf.ConfVars.HIVE_VECTORIZATION_GROUPBY_MAXENTRIES);
         this.minReductionHashAggr = HiveConf.getFloatVar(hconf,
             HiveConf.ConfVars.HIVEMAPAGGRHASHMINREDUCTION);
-          this.numRowsCompareHashAggr = HiveConf.getLongVar(hconf,
+          this.numRowsCompareHashAggr = HiveConf.getIntVar(hconf,
             HiveConf.ConfVars.HIVEGROUPBYMAPINTERVAL);
       } 
       else {

Modified: hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizedRowBatchCtx.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizedRowBatchCtx.java?rev=1583463&r1=1583462&r2=1583463&view=diff
==============================================================================
--- hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizedRowBatchCtx.java (original)
+++ hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizedRowBatchCtx.java Mon Mar 31 21:04:07 2014
@@ -18,6 +18,8 @@
 package org.apache.hadoop.hive.ql.exec.vector;
 
 import java.io.IOException;
+import java.sql.Date;
+import java.sql.Timestamp;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.LinkedHashMap;
@@ -28,6 +30,8 @@ import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.common.type.Decimal128;
+import org.apache.hadoop.hive.common.type.HiveDecimal;
 import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants;
 import org.apache.hadoop.hive.ql.exec.Utilities;
 import org.apache.hadoop.hive.ql.io.HiveFileFormatUtils;
@@ -38,12 +42,17 @@ import org.apache.hadoop.hive.serde2.Col
 import org.apache.hadoop.hive.serde2.Deserializer;
 import org.apache.hadoop.hive.serde2.SerDeException;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorConverters;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorConverters.Converter;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory;
 import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector.PrimitiveCategory;
 import org.apache.hadoop.hive.serde2.objectinspector.StructField;
 import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
 import org.apache.hadoop.hive.serde2.typeinfo.DecimalTypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.mapred.FileSplit;
 
@@ -54,7 +63,7 @@ import org.apache.hadoop.mapred.FileSpli
  * with the partition column.
  */
 public class VectorizedRowBatchCtx {
-
+  
   // OI for raw row data (EG without partition cols)
   private StructObjectInspector rawRowOI;
 
@@ -65,8 +74,11 @@ public class VectorizedRowBatchCtx {
   private Deserializer deserializer;
 
   // Hash map of partition values. Key=TblColName value=PartitionValue
-  private Map<String, String> partitionValues;
-
+  private Map<String, Object> partitionValues;
+  
+  //partition types
+  private Map<String, PrimitiveCategory> partitionTypes;  
+  
   // Column projection list - List of column indexes to include. This
   // list does not contain partition columns
   private List<Integer> colsToInclude;
@@ -86,11 +98,13 @@ public class VectorizedRowBatchCtx {
    *          Hash map of partition values. Key=TblColName value=PartitionValue
    */
   public VectorizedRowBatchCtx(StructObjectInspector rawRowOI, StructObjectInspector rowOI,
-      Deserializer deserializer, Map<String, String> partitionValues) {
+      Deserializer deserializer, Map<String, Object> partitionValues, 
+      Map<String, PrimitiveCategory> partitionTypes) {
     this.rowOI = rowOI;
     this.rawRowOI = rawRowOI;
     this.deserializer = deserializer;
     this.partitionValues = partitionValues;
+    this.partitionTypes = partitionTypes;
   }
 
   /**
@@ -173,25 +187,44 @@ public class VectorizedRowBatchCtx {
       // raw row object inspector (row with out partition col)
       LinkedHashMap<String, String> partSpec = part.getPartSpec();
       String[] partKeys = pcols.trim().split("/");
+      String pcolTypes = partProps.getProperty(hive_metastoreConstants.META_TABLE_PARTITION_COLUMN_TYPES);      
+      String[] partKeyTypes = pcolTypes.trim().split(":");      
+      
+      if (partKeys.length  > partKeyTypes.length) {
+        throw new HiveException("Internal error : partKeys length, " +partKeys.length +
+                " greater than partKeyTypes length, " + partKeyTypes.length);
+      }
+      
       List<String> partNames = new ArrayList<String>(partKeys.length);
-      partitionValues = new LinkedHashMap<String, String>();
-      List<ObjectInspector> partObjectInspectors = new ArrayList<ObjectInspector>(
-          partKeys.length);
+      List<ObjectInspector> partObjectInspectors = new ArrayList<ObjectInspector>(partKeys.length);
+      partitionValues = new LinkedHashMap<String, Object>();
+      partitionTypes = new LinkedHashMap<String, PrimitiveCategory>();
       for (int i = 0; i < partKeys.length; i++) {
         String key = partKeys[i];
         partNames.add(key);
+        ObjectInspector objectInspector = null;
+        Object objectVal; 
         if (partSpec == null) {
           // for partitionless table, initialize partValue to empty string.
           // We can have partitionless table even if we have partition keys
           // when there is only only partition selected and the partition key is not
           // part of the projection/include list.
-          partitionValues.put(key, "");
+          objectVal = null;
+          objectInspector = PrimitiveObjectInspectorFactory.javaStringObjectInspector;
+          partitionTypes.put(key, PrimitiveCategory.STRING);       
         } else {
-          partitionValues.put(key, partSpec.get(key));
+          // Create a Standard java object Inspector
+          objectInspector = TypeInfoUtils.getStandardJavaObjectInspectorFromTypeInfo(
+              TypeInfoFactory.getPrimitiveTypeInfo(partKeyTypes[i]));
+          objectVal = 
+              ObjectInspectorConverters.
+              getConverter(PrimitiveObjectInspectorFactory.
+                  javaStringObjectInspector, objectInspector).
+                  convert(partSpec.get(key));              
+          partitionTypes.put(key, TypeInfoFactory.getPrimitiveTypeInfo(partKeyTypes[i]).getPrimitiveCategory());
         }
-
-        partObjectInspectors
-            .add(PrimitiveObjectInspectorFactory.writableStringObjectInspector);
+        partitionValues.put(key, objectVal);
+        partObjectInspectors.add(objectInspector);
       }
 
       // Create partition OI
@@ -213,7 +246,7 @@ public class VectorizedRowBatchCtx {
 
     colsToInclude = ColumnProjectionUtils.getReadColumnIDs(hiveConf);
   }
-
+  
   /**
    * Creates a Vectorized row batch and the column vectors.
    *
@@ -274,8 +307,7 @@ public class VectorizedRowBatchCtx {
               + foi.getCategory());
         default:
           throw new HiveException("Unknown ObjectInspector category!");
-
-        }
+        }    
       }
     }
     result.numCols = fieldRefs.size();
@@ -334,7 +366,7 @@ public class VectorizedRowBatchCtx {
     }
     throw new HiveException("Not able to find column name in row object inspector");
   }
-
+  
   /**
    * Add the partition values to the batch
    *
@@ -344,17 +376,165 @@ public class VectorizedRowBatchCtx {
   public void addPartitionColsToBatch(VectorizedRowBatch batch) throws HiveException
   {
     int colIndex;
-    String value;
-    BytesColumnVector bcv;
+    Object value;
+    PrimitiveCategory pCategory;
     if (partitionValues != null) {
       for (String key : partitionValues.keySet()) {
         colIndex = getColIndexBasedOnColName(key);
         value = partitionValues.get(key);
-        bcv = (BytesColumnVector) batch.cols[colIndex];
-        bcv.setRef(0, value.getBytes(), 0, value.length());
-        bcv.isRepeating = true;
-        bcv.isNull[0] = false;
-        bcv.noNulls = true;
+        pCategory = partitionTypes.get(key);
+        
+        switch (pCategory) {
+        case BOOLEAN: {
+          LongColumnVector lcv = (LongColumnVector) batch.cols[colIndex];
+          if (value == null) {
+            lcv.noNulls = false;
+            lcv.isNull[0] = true;
+            lcv.isRepeating = true;
+          } else { 
+            lcv.fill((Boolean)value == true ? 1 : 0);
+            lcv.isNull[0] = false;
+          }
+        }
+        break;          
+        
+        case BYTE: {
+          LongColumnVector lcv = (LongColumnVector) batch.cols[colIndex];
+          if (value == null) {
+            lcv.noNulls = false;
+            lcv.isNull[0] = true;
+            lcv.isRepeating = true;
+          } else { 
+            lcv.fill((Byte)value);
+            lcv.isNull[0] = false;
+          }
+        }
+        break;             
+        
+        case SHORT: {
+          LongColumnVector lcv = (LongColumnVector) batch.cols[colIndex];
+          if (value == null) {
+            lcv.noNulls = false;
+            lcv.isNull[0] = true;
+            lcv.isRepeating = true;
+          } else { 
+            lcv.fill((Short)value);
+            lcv.isNull[0] = false;
+          }
+        }
+        break;
+        
+        case INT: {
+          LongColumnVector lcv = (LongColumnVector) batch.cols[colIndex];
+          if (value == null) {
+            lcv.noNulls = false;
+            lcv.isNull[0] = true;
+            lcv.isRepeating = true;
+          } else { 
+            lcv.fill((Integer)value);
+            lcv.isNull[0] = false;
+          }          
+        }
+        break;
+        
+        case LONG: {
+          LongColumnVector lcv = (LongColumnVector) batch.cols[colIndex];
+          if (value == null) {
+            lcv.noNulls = false;
+            lcv.isNull[0] = true;
+            lcv.isRepeating = true;
+          } else { 
+            lcv.fill((Long)value);
+            lcv.isNull[0] = false;
+          }          
+        }
+        break;
+        
+        case DATE: {
+          LongColumnVector lcv = (LongColumnVector) batch.cols[colIndex];
+          if (value == null) {
+            lcv.noNulls = false;
+            lcv.isNull[0] = true;
+            lcv.isRepeating = true;
+          } else { 
+            lcv.fill(((Date)value).getTime());
+            lcv.isNull[0] = false;
+          }          
+        }
+        break;
+        
+        case TIMESTAMP: {
+          LongColumnVector lcv = (LongColumnVector) batch.cols[colIndex];
+          if (value == null) {
+            lcv.noNulls = false;
+            lcv.isNull[0] = true;
+            lcv.isRepeating = true;
+          } else { 
+            lcv.fill((long)(((Timestamp) value).getTime()));
+            lcv.isNull[0] = false;
+          }
+        }
+        break;
+        
+        case FLOAT: {
+          DoubleColumnVector dcv = (DoubleColumnVector) batch.cols[colIndex];
+          if (value == null) {
+            dcv.noNulls = false;
+            dcv.isNull[0] = true;
+            dcv.isRepeating = true;
+          } else {
+            dcv.fill((Float) value);
+            dcv.isNull[0] = false;
+          }          
+        }
+        break;
+        
+        case DOUBLE: {
+          DoubleColumnVector dcv = (DoubleColumnVector) batch.cols[colIndex];
+          if (value == null) {
+            dcv.noNulls = false;
+            dcv.isNull[0] = true;
+            dcv.isRepeating = true;
+          } else {
+            dcv.fill((Double) value);
+            dcv.isNull[0] = false;
+          }
+        }
+        break;
+        
+        case DECIMAL: {
+          DecimalColumnVector dv = (DecimalColumnVector) batch.cols[colIndex];
+          if (value == null) {
+            dv.noNulls = false;
+            dv.isNull[0] = true;
+            dv.isRepeating = true;
+          } else {
+            HiveDecimal hd = (HiveDecimal)(value);
+            dv.vector[0] = new Decimal128(hd.toString(), (short)hd.scale());
+            dv.isRepeating = true;
+            dv.isNull[0] = false;      
+          }
+        }
+        break;
+          
+        case STRING: {
+          BytesColumnVector bcv = (BytesColumnVector) batch.cols[colIndex];
+          String sVal = (String)value;
+          if (sVal == null) {
+            bcv.noNulls = false;
+            bcv.isNull[0] = true;
+            bcv.isRepeating = true;
+          } else {
+            bcv.fill(sVal.getBytes()); 
+            bcv.isNull[0] = false;
+          }
+        }
+        break;
+        
+        default:
+          throw new HiveException("Unable to recognize the partition type " + pCategory + 
+              " for column " + key);
+        }
       }
     }
   }

Modified: hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java?rev=1583463&r1=1583462&r2=1583463&view=diff
==============================================================================
--- hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java (original)
+++ hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java Mon Mar 31 21:04:07 2014
@@ -8589,10 +8589,8 @@ public class SemanticAnalyzer extends Ba
       // Finally add the partitioning columns
       for (FieldSchema part_col : tab.getPartCols()) {
         LOG.trace("Adding partition col: " + part_col);
-        // TODO: use the right type by calling part_col.getType() instead of
-        // String.class. See HIVE-3059.
         rwsch.put(alias, part_col.getName(), new ColumnInfo(part_col.getName(),
-            TypeInfoFactory.stringTypeInfo, alias, true));
+            TypeInfoFactory.getPrimitiveTypeInfo(part_col.getType()), alias, true));
       }
 
       // put all virutal columns in RowResolver.

Modified: hive/branches/branch-0.13/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorizedRowBatchCtx.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorizedRowBatchCtx.java?rev=1583463&r1=1583462&r2=1583463&view=diff
==============================================================================
--- hive/branches/branch-0.13/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorizedRowBatchCtx.java (original)
+++ hive/branches/branch-0.13/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorizedRowBatchCtx.java Mon Mar 31 21:04:07 2014
@@ -202,7 +202,7 @@ public class TestVectorizedRowBatchCtx {
     Assert.assertEquals("Field size should be 9", colCount, fieldRefs.size());
 
     // Create the context
-    VectorizedRowBatchCtx ctx = new VectorizedRowBatchCtx(oi, oi, serDe, null);
+    VectorizedRowBatchCtx ctx = new VectorizedRowBatchCtx(oi, oi, serDe, null, null);
     VectorizedRowBatch batch = ctx.createVectorizedRowBatch();
     VectorizedBatchUtil.SetNoNullFields(true, batch);
 

Modified: hive/branches/branch-0.13/ql/src/test/queries/clientpositive/alter_partition_coltype.q
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/test/queries/clientpositive/alter_partition_coltype.q?rev=1583463&r1=1583462&r2=1583463&view=diff
==============================================================================
--- hive/branches/branch-0.13/ql/src/test/queries/clientpositive/alter_partition_coltype.q (original)
+++ hive/branches/branch-0.13/ql/src/test/queries/clientpositive/alter_partition_coltype.q Mon Mar 31 21:04:07 2014
@@ -10,48 +10,48 @@ desc alter_coltype;
 select count(*) from alter_coltype where dt = '100x';
 
 -- alter partition key column data type for dt column.
-alter table alter_coltype partition column (dt int);
+-- alter table alter_coltype partition column (dt int);
 
 -- load a new partition using new data type.
-insert overwrite table alter_coltype partition(dt=10, ts='3.0') select * from src1;
+-- insert overwrite table alter_coltype partition(dt=10, ts='3.0') select * from src1;
 
 -- make sure the partition predicate still works. 
-select count(*) from alter_coltype where dt = '100x';
-explain extended select count(*) from alter_coltype where dt = '100x';
+-- select count(*) from alter_coltype where dt = '100x';
+-- explain extended select count(*) from alter_coltype where dt = '100x';
 
-select count(*) from alter_coltype where dt = '100';
+-- select count(*) from alter_coltype where dt = '100';
 
 -- alter partition key column data type for ts column.
-alter table alter_coltype partition column (ts double);
+-- alter table alter_coltype partition column (ts double);
 
-alter table alter_coltype partition column (dt string);
+-- alter table alter_coltype partition column (dt string);
 
 -- load a new partition using new data type.
-insert overwrite table alter_coltype partition(dt='100x', ts=3.0) select * from src1;
+-- insert overwrite table alter_coltype partition(dt='100x', ts=3.0) select * from src1;
 
 --  validate partition key column predicate can still work.
-select count(*) from alter_coltype where ts = '6:30pm';
-explain extended select count(*) from alter_coltype where ts = '6:30pm';
+-- select count(*) from alter_coltype where ts = '6:30pm';
+-- explain extended select count(*) from alter_coltype where ts = '6:30pm';
 
 --  validate partition key column predicate on two different partition column data type 
 --  can still work.
-select count(*) from alter_coltype where ts = 3.0 and dt=10;
-explain extended select count(*) from alter_coltype where ts = 3.0 and dt=10;
+-- select count(*) from alter_coltype where ts = 3.0 and dt=10;
+-- explain extended select count(*) from alter_coltype where ts = 3.0 and dt=10;
 
 -- query where multiple partition values (of different datatypes) are being selected 
-select key, value, dt, ts from alter_coltype where dt is not null;
-explain extended select key, value, dt, ts from alter_coltype where dt is not null;
+-- select key, value, dt, ts from alter_coltype where dt is not null;
+-- explain extended select key, value, dt, ts from alter_coltype where dt is not null;
 
-select count(*) from alter_coltype where ts = 3.0;
+-- select count(*) from alter_coltype where ts = 3.0;
 
 -- make sure the partition predicate still works. 
-select count(*) from alter_coltype where dt = '100x' or dt = '10';
-explain extended select count(*) from alter_coltype where dt = '100x' or dt = '10';
+-- select count(*) from alter_coltype where dt = '100x' or dt = '10';
+-- explain extended select count(*) from alter_coltype where dt = '100x' or dt = '10';
 
-desc alter_coltype;
-desc alter_coltype partition (dt='100x', ts='6:30pm');
-desc alter_coltype partition (dt='100x', ts=3.0);
-desc alter_coltype partition (dt=10, ts=3.0);
+-- desc alter_coltype;
+-- desc alter_coltype partition (dt='100x', ts='6:30pm');
+-- desc alter_coltype partition (dt='100x', ts=3.0);
+-- desc alter_coltype partition (dt=10, ts=3.0);
 
 drop table alter_coltype;
 

Modified: hive/branches/branch-0.13/ql/src/test/queries/clientpositive/pcr.q
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/test/queries/clientpositive/pcr.q?rev=1583463&r1=1583462&r2=1583463&view=diff
==============================================================================
--- hive/branches/branch-0.13/ql/src/test/queries/clientpositive/pcr.q (original)
+++ hive/branches/branch-0.13/ql/src/test/queries/clientpositive/pcr.q Mon Mar 31 21:04:07 2014
@@ -112,7 +112,7 @@ insert overwrite table pcr_foo partition
 insert overwrite table pcr_foo partition (ds=7) select * from src where key < 10 order by key;
 
 -- the condition is 'true' for all the 3 partitions (ds=3,5,7):
-select key, value, ds from pcr_foo where (ds % 2 == 1);
+select key, value, ds from pcr_foo where (ds % 2.0 == 1);
 
 -- the condition is 'true' for partitions (ds=3,5) but 'false' of partition ds=7:
 select key, value, ds from pcr_foo where (ds / 3 < 2);

Modified: hive/branches/branch-0.13/ql/src/test/results/clientpositive/alter_partition_coltype.q.out
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/test/results/clientpositive/alter_partition_coltype.q.out?rev=1583463&r1=1583462&r2=1583463&view=diff
==============================================================================
Files hive/branches/branch-0.13/ql/src/test/results/clientpositive/alter_partition_coltype.q.out (original) and hive/branches/branch-0.13/ql/src/test/results/clientpositive/alter_partition_coltype.q.out Mon Mar 31 21:04:07 2014 differ

Modified: hive/branches/branch-0.13/ql/src/test/results/clientpositive/annotate_stats_part.q.out
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/test/results/clientpositive/annotate_stats_part.q.out?rev=1583463&r1=1583462&r2=1583463&view=diff
==============================================================================
Files hive/branches/branch-0.13/ql/src/test/results/clientpositive/annotate_stats_part.q.out (original) and hive/branches/branch-0.13/ql/src/test/results/clientpositive/annotate_stats_part.q.out Mon Mar 31 21:04:07 2014 differ

Modified: hive/branches/branch-0.13/ql/src/test/results/clientpositive/auto_sortmerge_join_1.q.out
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/test/results/clientpositive/auto_sortmerge_join_1.q.out?rev=1583463&r1=1583462&r2=1583463&view=diff
==============================================================================
Files hive/branches/branch-0.13/ql/src/test/results/clientpositive/auto_sortmerge_join_1.q.out (original) and hive/branches/branch-0.13/ql/src/test/results/clientpositive/auto_sortmerge_join_1.q.out Mon Mar 31 21:04:07 2014 differ

Modified: hive/branches/branch-0.13/ql/src/test/results/clientpositive/auto_sortmerge_join_11.q.out
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/test/results/clientpositive/auto_sortmerge_join_11.q.out?rev=1583463&r1=1583462&r2=1583463&view=diff
==============================================================================
Files hive/branches/branch-0.13/ql/src/test/results/clientpositive/auto_sortmerge_join_11.q.out (original) and hive/branches/branch-0.13/ql/src/test/results/clientpositive/auto_sortmerge_join_11.q.out Mon Mar 31 21:04:07 2014 differ

Modified: hive/branches/branch-0.13/ql/src/test/results/clientpositive/auto_sortmerge_join_12.q.out
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/test/results/clientpositive/auto_sortmerge_join_12.q.out?rev=1583463&r1=1583462&r2=1583463&view=diff
==============================================================================
Files hive/branches/branch-0.13/ql/src/test/results/clientpositive/auto_sortmerge_join_12.q.out (original) and hive/branches/branch-0.13/ql/src/test/results/clientpositive/auto_sortmerge_join_12.q.out Mon Mar 31 21:04:07 2014 differ

Modified: hive/branches/branch-0.13/ql/src/test/results/clientpositive/auto_sortmerge_join_2.q.out
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/test/results/clientpositive/auto_sortmerge_join_2.q.out?rev=1583463&r1=1583462&r2=1583463&view=diff
==============================================================================
Files hive/branches/branch-0.13/ql/src/test/results/clientpositive/auto_sortmerge_join_2.q.out (original) and hive/branches/branch-0.13/ql/src/test/results/clientpositive/auto_sortmerge_join_2.q.out Mon Mar 31 21:04:07 2014 differ

Modified: hive/branches/branch-0.13/ql/src/test/results/clientpositive/auto_sortmerge_join_3.q.out
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/test/results/clientpositive/auto_sortmerge_join_3.q.out?rev=1583463&r1=1583462&r2=1583463&view=diff
==============================================================================
Files hive/branches/branch-0.13/ql/src/test/results/clientpositive/auto_sortmerge_join_3.q.out (original) and hive/branches/branch-0.13/ql/src/test/results/clientpositive/auto_sortmerge_join_3.q.out Mon Mar 31 21:04:07 2014 differ

Modified: hive/branches/branch-0.13/ql/src/test/results/clientpositive/auto_sortmerge_join_4.q.out
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/test/results/clientpositive/auto_sortmerge_join_4.q.out?rev=1583463&r1=1583462&r2=1583463&view=diff
==============================================================================
Files hive/branches/branch-0.13/ql/src/test/results/clientpositive/auto_sortmerge_join_4.q.out (original) and hive/branches/branch-0.13/ql/src/test/results/clientpositive/auto_sortmerge_join_4.q.out Mon Mar 31 21:04:07 2014 differ

Modified: hive/branches/branch-0.13/ql/src/test/results/clientpositive/auto_sortmerge_join_7.q.out
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/test/results/clientpositive/auto_sortmerge_join_7.q.out?rev=1583463&r1=1583462&r2=1583463&view=diff
==============================================================================
Files hive/branches/branch-0.13/ql/src/test/results/clientpositive/auto_sortmerge_join_7.q.out (original) and hive/branches/branch-0.13/ql/src/test/results/clientpositive/auto_sortmerge_join_7.q.out Mon Mar 31 21:04:07 2014 differ

Modified: hive/branches/branch-0.13/ql/src/test/results/clientpositive/auto_sortmerge_join_8.q.out
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/test/results/clientpositive/auto_sortmerge_join_8.q.out?rev=1583463&r1=1583462&r2=1583463&view=diff
==============================================================================
Files hive/branches/branch-0.13/ql/src/test/results/clientpositive/auto_sortmerge_join_8.q.out (original) and hive/branches/branch-0.13/ql/src/test/results/clientpositive/auto_sortmerge_join_8.q.out Mon Mar 31 21:04:07 2014 differ

Modified: hive/branches/branch-0.13/ql/src/test/results/clientpositive/bucket3.q.out
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/test/results/clientpositive/bucket3.q.out?rev=1583463&r1=1583462&r2=1583463&view=diff
==============================================================================
Files hive/branches/branch-0.13/ql/src/test/results/clientpositive/bucket3.q.out (original) and hive/branches/branch-0.13/ql/src/test/results/clientpositive/bucket3.q.out Mon Mar 31 21:04:07 2014 differ

Modified: hive/branches/branch-0.13/ql/src/test/results/clientpositive/bucketcontext_1.q.out
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/test/results/clientpositive/bucketcontext_1.q.out?rev=1583463&r1=1583462&r2=1583463&view=diff
==============================================================================
Files hive/branches/branch-0.13/ql/src/test/results/clientpositive/bucketcontext_1.q.out (original) and hive/branches/branch-0.13/ql/src/test/results/clientpositive/bucketcontext_1.q.out Mon Mar 31 21:04:07 2014 differ

Modified: hive/branches/branch-0.13/ql/src/test/results/clientpositive/bucketcontext_2.q.out
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/test/results/clientpositive/bucketcontext_2.q.out?rev=1583463&r1=1583462&r2=1583463&view=diff
==============================================================================
Files hive/branches/branch-0.13/ql/src/test/results/clientpositive/bucketcontext_2.q.out (original) and hive/branches/branch-0.13/ql/src/test/results/clientpositive/bucketcontext_2.q.out Mon Mar 31 21:04:07 2014 differ

Modified: hive/branches/branch-0.13/ql/src/test/results/clientpositive/bucketcontext_3.q.out
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/test/results/clientpositive/bucketcontext_3.q.out?rev=1583463&r1=1583462&r2=1583463&view=diff
==============================================================================
Files hive/branches/branch-0.13/ql/src/test/results/clientpositive/bucketcontext_3.q.out (original) and hive/branches/branch-0.13/ql/src/test/results/clientpositive/bucketcontext_3.q.out Mon Mar 31 21:04:07 2014 differ

Modified: hive/branches/branch-0.13/ql/src/test/results/clientpositive/bucketcontext_4.q.out
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/test/results/clientpositive/bucketcontext_4.q.out?rev=1583463&r1=1583462&r2=1583463&view=diff
==============================================================================
Files hive/branches/branch-0.13/ql/src/test/results/clientpositive/bucketcontext_4.q.out (original) and hive/branches/branch-0.13/ql/src/test/results/clientpositive/bucketcontext_4.q.out Mon Mar 31 21:04:07 2014 differ

Modified: hive/branches/branch-0.13/ql/src/test/results/clientpositive/bucketcontext_6.q.out
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/test/results/clientpositive/bucketcontext_6.q.out?rev=1583463&r1=1583462&r2=1583463&view=diff
==============================================================================
Files hive/branches/branch-0.13/ql/src/test/results/clientpositive/bucketcontext_6.q.out (original) and hive/branches/branch-0.13/ql/src/test/results/clientpositive/bucketcontext_6.q.out Mon Mar 31 21:04:07 2014 differ

Modified: hive/branches/branch-0.13/ql/src/test/results/clientpositive/bucketcontext_7.q.out
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/test/results/clientpositive/bucketcontext_7.q.out?rev=1583463&r1=1583462&r2=1583463&view=diff
==============================================================================
Files hive/branches/branch-0.13/ql/src/test/results/clientpositive/bucketcontext_7.q.out (original) and hive/branches/branch-0.13/ql/src/test/results/clientpositive/bucketcontext_7.q.out Mon Mar 31 21:04:07 2014 differ

Modified: hive/branches/branch-0.13/ql/src/test/results/clientpositive/bucketcontext_8.q.out
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/test/results/clientpositive/bucketcontext_8.q.out?rev=1583463&r1=1583462&r2=1583463&view=diff
==============================================================================
Files hive/branches/branch-0.13/ql/src/test/results/clientpositive/bucketcontext_8.q.out (original) and hive/branches/branch-0.13/ql/src/test/results/clientpositive/bucketcontext_8.q.out Mon Mar 31 21:04:07 2014 differ

Modified: hive/branches/branch-0.13/ql/src/test/results/clientpositive/bucketmapjoin1.q.out
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/test/results/clientpositive/bucketmapjoin1.q.out?rev=1583463&r1=1583462&r2=1583463&view=diff
==============================================================================
--- hive/branches/branch-0.13/ql/src/test/results/clientpositive/bucketmapjoin1.q.out (original)
+++ hive/branches/branch-0.13/ql/src/test/results/clientpositive/bucketmapjoin1.q.out Mon Mar 31 21:04:07 2014
@@ -496,6 +496,7 @@ STAGE PLANS:
                         numFiles 4
                         numRows 0
                         partition_columns ds
+                        partition_columns.types string
                         rawDataSize 0
                         serialization.ddl struct srcbucket_mapjoin_part { i32 key, string value}
                         serialization.format 1
@@ -515,6 +516,7 @@ STAGE PLANS:
 #### A masked pattern was here ####
                           name default.srcbucket_mapjoin_part
                           partition_columns ds
+                          partition_columns.types string
                           serialization.ddl struct srcbucket_mapjoin_part { i32 key, string value}
                           serialization.format 1
                           serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
@@ -1097,6 +1099,7 @@ STAGE PLANS:
               numFiles 4
               numRows 0
               partition_columns ds
+              partition_columns.types string
               rawDataSize 0
               serialization.ddl struct srcbucket_mapjoin_part { i32 key, string value}
               serialization.format 1
@@ -1116,6 +1119,7 @@ STAGE PLANS:
 #### A masked pattern was here ####
                 name default.srcbucket_mapjoin_part
                 partition_columns ds
+                partition_columns.types string
                 serialization.ddl struct srcbucket_mapjoin_part { i32 key, string value}
                 serialization.format 1
                 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe

Modified: hive/branches/branch-0.13/ql/src/test/results/clientpositive/bucketmapjoin10.q.out
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/test/results/clientpositive/bucketmapjoin10.q.out?rev=1583463&r1=1583462&r2=1583463&view=diff
==============================================================================
Files hive/branches/branch-0.13/ql/src/test/results/clientpositive/bucketmapjoin10.q.out (original) and hive/branches/branch-0.13/ql/src/test/results/clientpositive/bucketmapjoin10.q.out Mon Mar 31 21:04:07 2014 differ

Modified: hive/branches/branch-0.13/ql/src/test/results/clientpositive/bucketmapjoin11.q.out
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/test/results/clientpositive/bucketmapjoin11.q.out?rev=1583463&r1=1583462&r2=1583463&view=diff
==============================================================================
Files hive/branches/branch-0.13/ql/src/test/results/clientpositive/bucketmapjoin11.q.out (original) and hive/branches/branch-0.13/ql/src/test/results/clientpositive/bucketmapjoin11.q.out Mon Mar 31 21:04:07 2014 differ

Modified: hive/branches/branch-0.13/ql/src/test/results/clientpositive/bucketmapjoin12.q.out
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/test/results/clientpositive/bucketmapjoin12.q.out?rev=1583463&r1=1583462&r2=1583463&view=diff
==============================================================================
Files hive/branches/branch-0.13/ql/src/test/results/clientpositive/bucketmapjoin12.q.out (original) and hive/branches/branch-0.13/ql/src/test/results/clientpositive/bucketmapjoin12.q.out Mon Mar 31 21:04:07 2014 differ

Modified: hive/branches/branch-0.13/ql/src/test/results/clientpositive/bucketmapjoin13.q.out
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/test/results/clientpositive/bucketmapjoin13.q.out?rev=1583463&r1=1583462&r2=1583463&view=diff
==============================================================================
Files hive/branches/branch-0.13/ql/src/test/results/clientpositive/bucketmapjoin13.q.out (original) and hive/branches/branch-0.13/ql/src/test/results/clientpositive/bucketmapjoin13.q.out Mon Mar 31 21:04:07 2014 differ

Modified: hive/branches/branch-0.13/ql/src/test/results/clientpositive/bucketmapjoin2.q.out
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/test/results/clientpositive/bucketmapjoin2.q.out?rev=1583463&r1=1583462&r2=1583463&view=diff
==============================================================================
Files hive/branches/branch-0.13/ql/src/test/results/clientpositive/bucketmapjoin2.q.out (original) and hive/branches/branch-0.13/ql/src/test/results/clientpositive/bucketmapjoin2.q.out Mon Mar 31 21:04:07 2014 differ

Modified: hive/branches/branch-0.13/ql/src/test/results/clientpositive/bucketmapjoin3.q.out
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/test/results/clientpositive/bucketmapjoin3.q.out?rev=1583463&r1=1583462&r2=1583463&view=diff
==============================================================================
Files hive/branches/branch-0.13/ql/src/test/results/clientpositive/bucketmapjoin3.q.out (original) and hive/branches/branch-0.13/ql/src/test/results/clientpositive/bucketmapjoin3.q.out Mon Mar 31 21:04:07 2014 differ

Modified: hive/branches/branch-0.13/ql/src/test/results/clientpositive/bucketmapjoin5.q.out
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/test/results/clientpositive/bucketmapjoin5.q.out?rev=1583463&r1=1583462&r2=1583463&view=diff
==============================================================================
Files hive/branches/branch-0.13/ql/src/test/results/clientpositive/bucketmapjoin5.q.out (original) and hive/branches/branch-0.13/ql/src/test/results/clientpositive/bucketmapjoin5.q.out Mon Mar 31 21:04:07 2014 differ

Modified: hive/branches/branch-0.13/ql/src/test/results/clientpositive/bucketmapjoin7.q.out
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/test/results/clientpositive/bucketmapjoin7.q.out?rev=1583463&r1=1583462&r2=1583463&view=diff
==============================================================================
Files hive/branches/branch-0.13/ql/src/test/results/clientpositive/bucketmapjoin7.q.out (original) and hive/branches/branch-0.13/ql/src/test/results/clientpositive/bucketmapjoin7.q.out Mon Mar 31 21:04:07 2014 differ

Modified: hive/branches/branch-0.13/ql/src/test/results/clientpositive/bucketmapjoin8.q.out
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/test/results/clientpositive/bucketmapjoin8.q.out?rev=1583463&r1=1583462&r2=1583463&view=diff
==============================================================================
Files hive/branches/branch-0.13/ql/src/test/results/clientpositive/bucketmapjoin8.q.out (original) and hive/branches/branch-0.13/ql/src/test/results/clientpositive/bucketmapjoin8.q.out Mon Mar 31 21:04:07 2014 differ

Modified: hive/branches/branch-0.13/ql/src/test/results/clientpositive/bucketmapjoin9.q.out
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/test/results/clientpositive/bucketmapjoin9.q.out?rev=1583463&r1=1583462&r2=1583463&view=diff
==============================================================================
Files hive/branches/branch-0.13/ql/src/test/results/clientpositive/bucketmapjoin9.q.out (original) and hive/branches/branch-0.13/ql/src/test/results/clientpositive/bucketmapjoin9.q.out Mon Mar 31 21:04:07 2014 differ

Modified: hive/branches/branch-0.13/ql/src/test/results/clientpositive/bucketmapjoin_negative.q.out
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/test/results/clientpositive/bucketmapjoin_negative.q.out?rev=1583463&r1=1583462&r2=1583463&view=diff
==============================================================================
Files hive/branches/branch-0.13/ql/src/test/results/clientpositive/bucketmapjoin_negative.q.out (original) and hive/branches/branch-0.13/ql/src/test/results/clientpositive/bucketmapjoin_negative.q.out Mon Mar 31 21:04:07 2014 differ

Modified: hive/branches/branch-0.13/ql/src/test/results/clientpositive/bucketmapjoin_negative2.q.out
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/test/results/clientpositive/bucketmapjoin_negative2.q.out?rev=1583463&r1=1583462&r2=1583463&view=diff
==============================================================================
Files hive/branches/branch-0.13/ql/src/test/results/clientpositive/bucketmapjoin_negative2.q.out (original) and hive/branches/branch-0.13/ql/src/test/results/clientpositive/bucketmapjoin_negative2.q.out Mon Mar 31 21:04:07 2014 differ

Modified: hive/branches/branch-0.13/ql/src/test/results/clientpositive/columnstats_partlvl.q.out
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/test/results/clientpositive/columnstats_partlvl.q.out?rev=1583463&r1=1583462&r2=1583463&view=diff
==============================================================================
Files hive/branches/branch-0.13/ql/src/test/results/clientpositive/columnstats_partlvl.q.out (original) and hive/branches/branch-0.13/ql/src/test/results/clientpositive/columnstats_partlvl.q.out Mon Mar 31 21:04:07 2014 differ

Modified: hive/branches/branch-0.13/ql/src/test/results/clientpositive/combine2_hadoop20.q.out
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/test/results/clientpositive/combine2_hadoop20.q.out?rev=1583463&r1=1583462&r2=1583463&view=diff
==============================================================================
--- hive/branches/branch-0.13/ql/src/test/results/clientpositive/combine2_hadoop20.q.out (original)
+++ hive/branches/branch-0.13/ql/src/test/results/clientpositive/combine2_hadoop20.q.out Mon Mar 31 21:04:07 2014
@@ -263,6 +263,7 @@ STAGE PLANS:
               numFiles 1
               numRows 1
               partition_columns value
+              partition_columns.types string
               rawDataSize 2
               serialization.ddl struct combine2 { string key}
               serialization.format 1
@@ -281,6 +282,7 @@ STAGE PLANS:
 #### A masked pattern was here ####
                 name default.combine2
                 partition_columns value
+                partition_columns.types string
                 serialization.ddl struct combine2 { string key}
                 serialization.format 1
                 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
@@ -306,6 +308,7 @@ STAGE PLANS:
               numFiles 1
               numRows 3
               partition_columns value
+              partition_columns.types string
               rawDataSize 3
               serialization.ddl struct combine2 { string key}
               serialization.format 1
@@ -324,6 +327,7 @@ STAGE PLANS:
 #### A masked pattern was here ####
                 name default.combine2
                 partition_columns value
+                partition_columns.types string
                 serialization.ddl struct combine2 { string key}
                 serialization.format 1
                 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
@@ -349,6 +353,7 @@ STAGE PLANS:
               numFiles 1
               numRows 1
               partition_columns value
+              partition_columns.types string
               rawDataSize 1
               serialization.ddl struct combine2 { string key}
               serialization.format 1
@@ -367,6 +372,7 @@ STAGE PLANS:
 #### A masked pattern was here ####
                 name default.combine2
                 partition_columns value
+                partition_columns.types string
                 serialization.ddl struct combine2 { string key}
                 serialization.format 1
                 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
@@ -392,6 +398,7 @@ STAGE PLANS:
               numFiles 1
               numRows 1
               partition_columns value
+              partition_columns.types string
               rawDataSize 1
               serialization.ddl struct combine2 { string key}
               serialization.format 1
@@ -410,6 +417,7 @@ STAGE PLANS:
 #### A masked pattern was here ####
                 name default.combine2
                 partition_columns value
+                partition_columns.types string
                 serialization.ddl struct combine2 { string key}
                 serialization.format 1
                 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
@@ -435,6 +443,7 @@ STAGE PLANS:
               numFiles 1
               numRows 3
               partition_columns value
+              partition_columns.types string
               rawDataSize 3
               serialization.ddl struct combine2 { string key}
               serialization.format 1
@@ -453,6 +462,7 @@ STAGE PLANS:
 #### A masked pattern was here ####
                 name default.combine2
                 partition_columns value
+                partition_columns.types string
                 serialization.ddl struct combine2 { string key}
                 serialization.format 1
                 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
@@ -478,6 +488,7 @@ STAGE PLANS:
               numFiles 1
               numRows 1
               partition_columns value
+              partition_columns.types string
               rawDataSize 1
               serialization.ddl struct combine2 { string key}
               serialization.format 1
@@ -496,6 +507,7 @@ STAGE PLANS:
 #### A masked pattern was here ####
                 name default.combine2
                 partition_columns value
+                partition_columns.types string
                 serialization.ddl struct combine2 { string key}
                 serialization.format 1
                 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
@@ -521,6 +533,7 @@ STAGE PLANS:
               numFiles 1
               numRows 1
               partition_columns value
+              partition_columns.types string
               rawDataSize 1
               serialization.ddl struct combine2 { string key}
               serialization.format 1
@@ -539,6 +552,7 @@ STAGE PLANS:
 #### A masked pattern was here ####
                 name default.combine2
                 partition_columns value
+                partition_columns.types string
                 serialization.ddl struct combine2 { string key}
                 serialization.format 1
                 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
@@ -564,6 +578,7 @@ STAGE PLANS:
               numFiles 1
               numRows 1
               partition_columns value
+              partition_columns.types string
               rawDataSize 2
               serialization.ddl struct combine2 { string key}
               serialization.format 1
@@ -582,6 +597,7 @@ STAGE PLANS:
 #### A masked pattern was here ####
                 name default.combine2
                 partition_columns value
+                partition_columns.types string
                 serialization.ddl struct combine2 { string key}
                 serialization.format 1
                 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe

Modified: hive/branches/branch-0.13/ql/src/test/results/clientpositive/dynamic_partition_skip_default.q.out
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/test/results/clientpositive/dynamic_partition_skip_default.q.out?rev=1583463&r1=1583462&r2=1583463&view=diff
==============================================================================
--- hive/branches/branch-0.13/ql/src/test/results/clientpositive/dynamic_partition_skip_default.q.out (original)
+++ hive/branches/branch-0.13/ql/src/test/results/clientpositive/dynamic_partition_skip_default.q.out Mon Mar 31 21:04:07 2014
@@ -141,6 +141,7 @@ STAGE PLANS:
               numFiles 1
               numRows 1
               partition_columns partcol1/partcol2
+              partition_columns.types string:string
               rawDataSize 1
               serialization.ddl struct dynamic_part_table { string intcol}
               serialization.format 1
@@ -159,6 +160,7 @@ STAGE PLANS:
 #### A masked pattern was here ####
                 name default.dynamic_part_table
                 partition_columns partcol1/partcol2
+                partition_columns.types string:string
                 serialization.ddl struct dynamic_part_table { string intcol}
                 serialization.format 1
                 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
@@ -267,6 +269,7 @@ STAGE PLANS:
               numFiles 1
               numRows 1
               partition_columns partcol1/partcol2
+              partition_columns.types string:string
               rawDataSize 1
               serialization.ddl struct dynamic_part_table { string intcol}
               serialization.format 1
@@ -285,6 +288,7 @@ STAGE PLANS:
 #### A masked pattern was here ####
                 name default.dynamic_part_table
                 partition_columns partcol1/partcol2
+                partition_columns.types string:string
                 serialization.ddl struct dynamic_part_table { string intcol}
                 serialization.format 1
                 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
@@ -403,6 +407,7 @@ STAGE PLANS:
               numFiles 1
               numRows 1
               partition_columns partcol1/partcol2
+              partition_columns.types string:string
               rawDataSize 1
               serialization.ddl struct dynamic_part_table { string intcol}
               serialization.format 1
@@ -421,6 +426,7 @@ STAGE PLANS:
 #### A masked pattern was here ####
                 name default.dynamic_part_table
                 partition_columns partcol1/partcol2
+                partition_columns.types string:string
                 serialization.ddl struct dynamic_part_table { string intcol}
                 serialization.format 1
                 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
@@ -447,6 +453,7 @@ STAGE PLANS:
               numFiles 1
               numRows 1
               partition_columns partcol1/partcol2
+              partition_columns.types string:string
               rawDataSize 1
               serialization.ddl struct dynamic_part_table { string intcol}
               serialization.format 1
@@ -465,6 +472,7 @@ STAGE PLANS:
 #### A masked pattern was here ####
                 name default.dynamic_part_table
                 partition_columns partcol1/partcol2
+                partition_columns.types string:string
                 serialization.ddl struct dynamic_part_table { string intcol}
                 serialization.format 1
                 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe

Modified: hive/branches/branch-0.13/ql/src/test/results/clientpositive/filter_join_breaktask.q.out
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/test/results/clientpositive/filter_join_breaktask.q.out?rev=1583463&r1=1583462&r2=1583463&view=diff
==============================================================================
Files hive/branches/branch-0.13/ql/src/test/results/clientpositive/filter_join_breaktask.q.out (original) and hive/branches/branch-0.13/ql/src/test/results/clientpositive/filter_join_breaktask.q.out Mon Mar 31 21:04:07 2014 differ

Modified: hive/branches/branch-0.13/ql/src/test/results/clientpositive/filter_numeric.q.out
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/test/results/clientpositive/filter_numeric.q.out?rev=1583463&r1=1583462&r2=1583463&view=diff
==============================================================================
--- hive/branches/branch-0.13/ql/src/test/results/clientpositive/filter_numeric.q.out (original)
+++ hive/branches/branch-0.13/ql/src/test/results/clientpositive/filter_numeric.q.out Mon Mar 31 21:04:07 2014
@@ -74,7 +74,7 @@ STAGE PLANS:
             alias: partint
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Select Operator
-              expressions: key (type: string), value (type: string), hr (type: string)
+              expressions: key (type: string), value (type: string), hr (type: int)
               outputColumnNames: _col0, _col1, _col2
               Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               File Output Operator
@@ -623,7 +623,7 @@ STAGE PLANS:
             alias: partint
             Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE
             Select Operator
-              expressions: key (type: string), value (type: string), hr (type: string)
+              expressions: key (type: string), value (type: string), hr (type: int)
               outputColumnNames: _col0, _col1, _col2
               Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE
               File Output Operator
@@ -1674,7 +1674,7 @@ STAGE PLANS:
             alias: partint
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Select Operator
-              expressions: key (type: string), value (type: string), hr (type: string)
+              expressions: key (type: string), value (type: string), hr (type: int)
               outputColumnNames: _col0, _col1, _col2
               Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               File Output Operator
@@ -2223,7 +2223,7 @@ STAGE PLANS:
             alias: partint
             Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE
             Select Operator
-              expressions: key (type: string), value (type: string), hr (type: string)
+              expressions: key (type: string), value (type: string), hr (type: int)
               outputColumnNames: _col0, _col1, _col2
               Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE
               File Output Operator

Modified: hive/branches/branch-0.13/ql/src/test/results/clientpositive/groupby_map_ppr.q.out
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/test/results/clientpositive/groupby_map_ppr.q.out?rev=1583463&r1=1583462&r2=1583463&view=diff
==============================================================================
Files hive/branches/branch-0.13/ql/src/test/results/clientpositive/groupby_map_ppr.q.out (original) and hive/branches/branch-0.13/ql/src/test/results/clientpositive/groupby_map_ppr.q.out Mon Mar 31 21:04:07 2014 differ

Modified: hive/branches/branch-0.13/ql/src/test/results/clientpositive/groupby_map_ppr_multi_distinct.q.out
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/test/results/clientpositive/groupby_map_ppr_multi_distinct.q.out?rev=1583463&r1=1583462&r2=1583463&view=diff
==============================================================================
Files hive/branches/branch-0.13/ql/src/test/results/clientpositive/groupby_map_ppr_multi_distinct.q.out (original) and hive/branches/branch-0.13/ql/src/test/results/clientpositive/groupby_map_ppr_multi_distinct.q.out Mon Mar 31 21:04:07 2014 differ

Modified: hive/branches/branch-0.13/ql/src/test/results/clientpositive/groupby_ppr.q.out
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/test/results/clientpositive/groupby_ppr.q.out?rev=1583463&r1=1583462&r2=1583463&view=diff
==============================================================================
Files hive/branches/branch-0.13/ql/src/test/results/clientpositive/groupby_ppr.q.out (original) and hive/branches/branch-0.13/ql/src/test/results/clientpositive/groupby_ppr.q.out Mon Mar 31 21:04:07 2014 differ

Modified: hive/branches/branch-0.13/ql/src/test/results/clientpositive/groupby_ppr_multi_distinct.q.out
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/test/results/clientpositive/groupby_ppr_multi_distinct.q.out?rev=1583463&r1=1583462&r2=1583463&view=diff
==============================================================================
Files hive/branches/branch-0.13/ql/src/test/results/clientpositive/groupby_ppr_multi_distinct.q.out (original) and hive/branches/branch-0.13/ql/src/test/results/clientpositive/groupby_ppr_multi_distinct.q.out Mon Mar 31 21:04:07 2014 differ

Modified: hive/branches/branch-0.13/ql/src/test/results/clientpositive/groupby_sort_6.q.out
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/test/results/clientpositive/groupby_sort_6.q.out?rev=1583463&r1=1583462&r2=1583463&view=diff
==============================================================================
Files hive/branches/branch-0.13/ql/src/test/results/clientpositive/groupby_sort_6.q.out (original) and hive/branches/branch-0.13/ql/src/test/results/clientpositive/groupby_sort_6.q.out Mon Mar 31 21:04:07 2014 differ

Modified: hive/branches/branch-0.13/ql/src/test/results/clientpositive/input23.q.out
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/test/results/clientpositive/input23.q.out?rev=1583463&r1=1583462&r2=1583463&view=diff
==============================================================================
Files hive/branches/branch-0.13/ql/src/test/results/clientpositive/input23.q.out (original) and hive/branches/branch-0.13/ql/src/test/results/clientpositive/input23.q.out Mon Mar 31 21:04:07 2014 differ

Modified: hive/branches/branch-0.13/ql/src/test/results/clientpositive/input42.q.out
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/test/results/clientpositive/input42.q.out?rev=1583463&r1=1583462&r2=1583463&view=diff
==============================================================================
Files hive/branches/branch-0.13/ql/src/test/results/clientpositive/input42.q.out (original) and hive/branches/branch-0.13/ql/src/test/results/clientpositive/input42.q.out Mon Mar 31 21:04:07 2014 differ

Modified: hive/branches/branch-0.13/ql/src/test/results/clientpositive/input_part1.q.out
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/test/results/clientpositive/input_part1.q.out?rev=1583463&r1=1583462&r2=1583463&view=diff
==============================================================================
Files hive/branches/branch-0.13/ql/src/test/results/clientpositive/input_part1.q.out (original) and hive/branches/branch-0.13/ql/src/test/results/clientpositive/input_part1.q.out Mon Mar 31 21:04:07 2014 differ

Modified: hive/branches/branch-0.13/ql/src/test/results/clientpositive/input_part2.q.out
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/test/results/clientpositive/input_part2.q.out?rev=1583463&r1=1583462&r2=1583463&view=diff
==============================================================================
Files hive/branches/branch-0.13/ql/src/test/results/clientpositive/input_part2.q.out (original) and hive/branches/branch-0.13/ql/src/test/results/clientpositive/input_part2.q.out Mon Mar 31 21:04:07 2014 differ

Modified: hive/branches/branch-0.13/ql/src/test/results/clientpositive/input_part7.q.out
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/test/results/clientpositive/input_part7.q.out?rev=1583463&r1=1583462&r2=1583463&view=diff
==============================================================================
Files hive/branches/branch-0.13/ql/src/test/results/clientpositive/input_part7.q.out (original) and hive/branches/branch-0.13/ql/src/test/results/clientpositive/input_part7.q.out Mon Mar 31 21:04:07 2014 differ

Modified: hive/branches/branch-0.13/ql/src/test/results/clientpositive/input_part9.q.out
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/test/results/clientpositive/input_part9.q.out?rev=1583463&r1=1583462&r2=1583463&view=diff
==============================================================================
Files hive/branches/branch-0.13/ql/src/test/results/clientpositive/input_part9.q.out (original) and hive/branches/branch-0.13/ql/src/test/results/clientpositive/input_part9.q.out Mon Mar 31 21:04:07 2014 differ

Modified: hive/branches/branch-0.13/ql/src/test/results/clientpositive/join26.q.out
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/test/results/clientpositive/join26.q.out?rev=1583463&r1=1583462&r2=1583463&view=diff
==============================================================================
Files hive/branches/branch-0.13/ql/src/test/results/clientpositive/join26.q.out (original) and hive/branches/branch-0.13/ql/src/test/results/clientpositive/join26.q.out Mon Mar 31 21:04:07 2014 differ

Modified: hive/branches/branch-0.13/ql/src/test/results/clientpositive/join32.q.out
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/test/results/clientpositive/join32.q.out?rev=1583463&r1=1583462&r2=1583463&view=diff
==============================================================================
Files hive/branches/branch-0.13/ql/src/test/results/clientpositive/join32.q.out (original) and hive/branches/branch-0.13/ql/src/test/results/clientpositive/join32.q.out Mon Mar 31 21:04:07 2014 differ

Modified: hive/branches/branch-0.13/ql/src/test/results/clientpositive/join32_lessSize.q.out
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/test/results/clientpositive/join32_lessSize.q.out?rev=1583463&r1=1583462&r2=1583463&view=diff
==============================================================================
Files hive/branches/branch-0.13/ql/src/test/results/clientpositive/join32_lessSize.q.out (original) and hive/branches/branch-0.13/ql/src/test/results/clientpositive/join32_lessSize.q.out Mon Mar 31 21:04:07 2014 differ

Modified: hive/branches/branch-0.13/ql/src/test/results/clientpositive/join33.q.out
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/test/results/clientpositive/join33.q.out?rev=1583463&r1=1583462&r2=1583463&view=diff
==============================================================================
Files hive/branches/branch-0.13/ql/src/test/results/clientpositive/join33.q.out (original) and hive/branches/branch-0.13/ql/src/test/results/clientpositive/join33.q.out Mon Mar 31 21:04:07 2014 differ

Modified: hive/branches/branch-0.13/ql/src/test/results/clientpositive/join9.q.out
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/test/results/clientpositive/join9.q.out?rev=1583463&r1=1583462&r2=1583463&view=diff
==============================================================================
Files hive/branches/branch-0.13/ql/src/test/results/clientpositive/join9.q.out (original) and hive/branches/branch-0.13/ql/src/test/results/clientpositive/join9.q.out Mon Mar 31 21:04:07 2014 differ

Modified: hive/branches/branch-0.13/ql/src/test/results/clientpositive/join_map_ppr.q.out
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/test/results/clientpositive/join_map_ppr.q.out?rev=1583463&r1=1583462&r2=1583463&view=diff
==============================================================================
Files hive/branches/branch-0.13/ql/src/test/results/clientpositive/join_map_ppr.q.out (original) and hive/branches/branch-0.13/ql/src/test/results/clientpositive/join_map_ppr.q.out Mon Mar 31 21:04:07 2014 differ

Modified: hive/branches/branch-0.13/ql/src/test/results/clientpositive/list_bucket_dml_10.q.out
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/test/results/clientpositive/list_bucket_dml_10.q.out?rev=1583463&r1=1583462&r2=1583463&view=diff
==============================================================================
Files hive/branches/branch-0.13/ql/src/test/results/clientpositive/list_bucket_dml_10.q.out (original) and hive/branches/branch-0.13/ql/src/test/results/clientpositive/list_bucket_dml_10.q.out Mon Mar 31 21:04:07 2014 differ

Modified: hive/branches/branch-0.13/ql/src/test/results/clientpositive/load_dyn_part8.q.out
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/test/results/clientpositive/load_dyn_part8.q.out?rev=1583463&r1=1583462&r2=1583463&view=diff
==============================================================================
Files hive/branches/branch-0.13/ql/src/test/results/clientpositive/load_dyn_part8.q.out (original) and hive/branches/branch-0.13/ql/src/test/results/clientpositive/load_dyn_part8.q.out Mon Mar 31 21:04:07 2014 differ

Modified: hive/branches/branch-0.13/ql/src/test/results/clientpositive/louter_join_ppr.q.out
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/test/results/clientpositive/louter_join_ppr.q.out?rev=1583463&r1=1583462&r2=1583463&view=diff
==============================================================================
Files hive/branches/branch-0.13/ql/src/test/results/clientpositive/louter_join_ppr.q.out (original) and hive/branches/branch-0.13/ql/src/test/results/clientpositive/louter_join_ppr.q.out Mon Mar 31 21:04:07 2014 differ

Modified: hive/branches/branch-0.13/ql/src/test/results/clientpositive/merge3.q.out
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/test/results/clientpositive/merge3.q.out?rev=1583463&r1=1583462&r2=1583463&view=diff
==============================================================================
Files hive/branches/branch-0.13/ql/src/test/results/clientpositive/merge3.q.out (original) and hive/branches/branch-0.13/ql/src/test/results/clientpositive/merge3.q.out Mon Mar 31 21:04:07 2014 differ

Modified: hive/branches/branch-0.13/ql/src/test/results/clientpositive/metadataonly1.q.out
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/test/results/clientpositive/metadataonly1.q.out?rev=1583463&r1=1583462&r2=1583463&view=diff
==============================================================================
Files hive/branches/branch-0.13/ql/src/test/results/clientpositive/metadataonly1.q.out (original) and hive/branches/branch-0.13/ql/src/test/results/clientpositive/metadataonly1.q.out Mon Mar 31 21:04:07 2014 differ

Modified: hive/branches/branch-0.13/ql/src/test/results/clientpositive/outer_join_ppr.q.out
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/test/results/clientpositive/outer_join_ppr.q.out?rev=1583463&r1=1583462&r2=1583463&view=diff
==============================================================================
Files hive/branches/branch-0.13/ql/src/test/results/clientpositive/outer_join_ppr.q.out (original) and hive/branches/branch-0.13/ql/src/test/results/clientpositive/outer_join_ppr.q.out Mon Mar 31 21:04:07 2014 differ

Modified: hive/branches/branch-0.13/ql/src/test/results/clientpositive/pcr.q.out
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/test/results/clientpositive/pcr.q.out?rev=1583463&r1=1583462&r2=1583463&view=diff
==============================================================================
Files hive/branches/branch-0.13/ql/src/test/results/clientpositive/pcr.q.out (original) and hive/branches/branch-0.13/ql/src/test/results/clientpositive/pcr.q.out Mon Mar 31 21:04:07 2014 differ

Modified: hive/branches/branch-0.13/ql/src/test/results/clientpositive/ppd_union_view.q.out
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/test/results/clientpositive/ppd_union_view.q.out?rev=1583463&r1=1583462&r2=1583463&view=diff
==============================================================================
Files hive/branches/branch-0.13/ql/src/test/results/clientpositive/ppd_union_view.q.out (original) and hive/branches/branch-0.13/ql/src/test/results/clientpositive/ppd_union_view.q.out Mon Mar 31 21:04:07 2014 differ

Modified: hive/branches/branch-0.13/ql/src/test/results/clientpositive/ppd_vc.q.out
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/test/results/clientpositive/ppd_vc.q.out?rev=1583463&r1=1583462&r2=1583463&view=diff
==============================================================================
Files hive/branches/branch-0.13/ql/src/test/results/clientpositive/ppd_vc.q.out (original) and hive/branches/branch-0.13/ql/src/test/results/clientpositive/ppd_vc.q.out Mon Mar 31 21:04:07 2014 differ

Modified: hive/branches/branch-0.13/ql/src/test/results/clientpositive/ppr_allchildsarenull.q.out
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/test/results/clientpositive/ppr_allchildsarenull.q.out?rev=1583463&r1=1583462&r2=1583463&view=diff
==============================================================================
Files hive/branches/branch-0.13/ql/src/test/results/clientpositive/ppr_allchildsarenull.q.out (original) and hive/branches/branch-0.13/ql/src/test/results/clientpositive/ppr_allchildsarenull.q.out Mon Mar 31 21:04:07 2014 differ

Modified: hive/branches/branch-0.13/ql/src/test/results/clientpositive/push_or.q.out
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/test/results/clientpositive/push_or.q.out?rev=1583463&r1=1583462&r2=1583463&view=diff
==============================================================================
Files hive/branches/branch-0.13/ql/src/test/results/clientpositive/push_or.q.out (original) and hive/branches/branch-0.13/ql/src/test/results/clientpositive/push_or.q.out Mon Mar 31 21:04:07 2014 differ