You are viewing a plain text version of this content. The canonical link for it is here.
Posted to hcatalog-commits@incubator.apache.org by ha...@apache.org on 2011/12/07 07:30:09 UTC

svn commit: r1211316 - in /incubator/hcatalog/branches/branch-0.2: ./ src/java/org/apache/hcatalog/mapreduce/ src/java/org/apache/hcatalog/pig/ src/test/org/apache/hcatalog/pig/

Author: hashutosh
Date: Wed Dec  7 07:30:09 2011
New Revision: 1211316

URL: http://svn.apache.org/viewvc?rev=1211316&view=rev
Log:
HCATALOG-136: Pig storer for export/import does maintain partition order (Krishna Kumar via hashutosh)

Modified:
    incubator/hcatalog/branches/branch-0.2/CHANGES.txt
    incubator/hcatalog/branches/branch-0.2/src/java/org/apache/hcatalog/mapreduce/HCatEximOutputFormat.java
    incubator/hcatalog/branches/branch-0.2/src/java/org/apache/hcatalog/pig/HCatBaseStorer.java
    incubator/hcatalog/branches/branch-0.2/src/java/org/apache/hcatalog/pig/HCatEximStorer.java
    incubator/hcatalog/branches/branch-0.2/src/test/org/apache/hcatalog/pig/TestHCatEximStorer.java

Modified: incubator/hcatalog/branches/branch-0.2/CHANGES.txt
URL: http://svn.apache.org/viewvc/incubator/hcatalog/branches/branch-0.2/CHANGES.txt?rev=1211316&r1=1211315&r2=1211316&view=diff
==============================================================================
--- incubator/hcatalog/branches/branch-0.2/CHANGES.txt (original)
+++ incubator/hcatalog/branches/branch-0.2/CHANGES.txt Wed Dec  7 07:30:09 2011
@@ -81,6 +81,8 @@ Release 0.2.0 - Sept-2011
   OPTIMIZATIONS
 
   BUG FIXES
+    HCAT-136. Pig storer for export/import does maintain partition order (Krishna Kumar via hashutosh)
+
     HCAT-150. Updated documentation specifying the need to call HCatOutputFormat.setSchema() (chandec via toffer)
 
     HCAT-107. Better documentation for "hive.metastore.warehouse.dir" config (chandec via toffer)    

Modified: incubator/hcatalog/branches/branch-0.2/src/java/org/apache/hcatalog/mapreduce/HCatEximOutputFormat.java
URL: http://svn.apache.org/viewvc/incubator/hcatalog/branches/branch-0.2/src/java/org/apache/hcatalog/mapreduce/HCatEximOutputFormat.java?rev=1211316&r1=1211315&r2=1211316&view=diff
==============================================================================
--- incubator/hcatalog/branches/branch-0.2/src/java/org/apache/hcatalog/mapreduce/HCatEximOutputFormat.java (original)
+++ incubator/hcatalog/branches/branch-0.2/src/java/org/apache/hcatalog/mapreduce/HCatEximOutputFormat.java Wed Dec  7 07:30:09 2011
@@ -28,6 +28,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.metastore.Warehouse;
+import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.metastore.api.SerDeInfo;
 import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
@@ -138,8 +139,9 @@ public class HCatEximOutputFormat extend
     try {
       String partname = null;
       if ((partKeys != null) && !partKeys.isEmpty()) {
-        table.setPartitionKeys(HCatSchemaUtils.getFieldSchemas(partKeys));
-        partname = Warehouse.makePartPath(partSpec);
+        List<FieldSchema> partSchema = HCatSchemaUtils.getFieldSchemas(partKeys);
+        table.setPartitionKeys(partSchema);
+        partname = Warehouse.makePartName(partSchema, partitionValues);
       } else {
         partname = "data";
       }

Modified: incubator/hcatalog/branches/branch-0.2/src/java/org/apache/hcatalog/pig/HCatBaseStorer.java
URL: http://svn.apache.org/viewvc/incubator/hcatalog/branches/branch-0.2/src/java/org/apache/hcatalog/pig/HCatBaseStorer.java?rev=1211316&r1=1211315&r2=1211316&view=diff
==============================================================================
--- incubator/hcatalog/branches/branch-0.2/src/java/org/apache/hcatalog/pig/HCatBaseStorer.java (original)
+++ incubator/hcatalog/branches/branch-0.2/src/java/org/apache/hcatalog/pig/HCatBaseStorer.java Wed Dec  7 07:30:09 2011
@@ -62,6 +62,7 @@ public abstract class HCatBaseStorer ext
    *
    */
   protected static final String COMPUTED_OUTPUT_SCHEMA = "hcat.output.schema";
+  protected final List<String> partitionKeys;
   protected final Map<String,String> partitions;
   protected Schema pigSchema;
   private RecordWriter<WritableComparable<?>, HCatRecord> writer;
@@ -71,13 +72,16 @@ public abstract class HCatBaseStorer ext
 
   public HCatBaseStorer(String partSpecs, String schema) throws Exception {
 
+    partitionKeys = new ArrayList<String>();
     partitions = new HashMap<String, String>();
     if(partSpecs != null && !partSpecs.trim().isEmpty()){
       String[] partKVPs = partSpecs.split(",");
       for(String partKVP : partKVPs){
         String[] partKV = partKVP.split("=");
         if(partKV.length == 2) {
-          partitions.put(partKV[0].trim(), partKV[1].trim());
+          String partKey = partKV[0].trim(); 
+          partitionKeys.add(partKey);
+          partitions.put(partKey, partKV[1].trim());
         } else {
           throw new FrontendException("Invalid partition column specification. "+partSpecs, PigHCatUtil.PIG_EXCEPTION_CODE);
         }

Modified: incubator/hcatalog/branches/branch-0.2/src/java/org/apache/hcatalog/pig/HCatEximStorer.java
URL: http://svn.apache.org/viewvc/incubator/hcatalog/branches/branch-0.2/src/java/org/apache/hcatalog/pig/HCatEximStorer.java?rev=1211316&r1=1211315&r2=1211316&view=diff
==============================================================================
--- incubator/hcatalog/branches/branch-0.2/src/java/org/apache/hcatalog/pig/HCatEximStorer.java (original)
+++ incubator/hcatalog/branches/branch-0.2/src/java/org/apache/hcatalog/pig/HCatEximStorer.java Wed Dec  7 07:30:09 2011
@@ -107,7 +107,7 @@ public class HCatEximStorer extends HCat
 
       List<HCatFieldSchema> hcatFields = new ArrayList<HCatFieldSchema>();
       List<String> partVals = new ArrayList<String>();
-      for (String key : partitions.keySet()) {
+      for (String key : partitionKeys) {
         hcatFields.add(new HCatFieldSchema(key, HCatFieldSchema.Type.STRING, ""));
         partVals.add(partitions.get(key));
       }

Modified: incubator/hcatalog/branches/branch-0.2/src/test/org/apache/hcatalog/pig/TestHCatEximStorer.java
URL: http://svn.apache.org/viewvc/incubator/hcatalog/branches/branch-0.2/src/test/org/apache/hcatalog/pig/TestHCatEximStorer.java?rev=1211316&r1=1211315&r2=1211316&view=diff
==============================================================================
--- incubator/hcatalog/branches/branch-0.2/src/test/org/apache/hcatalog/pig/TestHCatEximStorer.java (original)
+++ incubator/hcatalog/branches/branch-0.2/src/test/org/apache/hcatalog/pig/TestHCatEximStorer.java Wed Dec  7 07:30:09 2011
@@ -196,6 +196,64 @@ public class TestHCatEximStorer extends 
         partition.getParameters().get(HCatConstants.HCAT_OSD_CLASS));
   }
 
+  public void testStorePartTable_state_country() throws Exception {
+	    populateDataFile();
+	    PigServer server = new PigServer(ExecType.LOCAL, props);
+	    UDFContext.getUDFContext().setClientSystemProps();
+	    server.setBatchOn();
+	    server.registerQuery("A = load '" + fqdataLocation + "' using PigStorage(',') as (emp_id:int, emp_name:chararray, emp_dob:chararray, emp_sex:chararray);");
+	    server.registerQuery("store A into '" + PARTITIONED_TABLE
+	        + "' using org.apache.hcatalog.pig.HCatEximStorer('" + fqexportLocation + "', 'emp_state=tn,emp_country=in');");
+	    server.executeBatch();
+
+	    FileSystem fs = cluster.getFileSystem();
+
+	    System.out.println("Filesystem class : " + cluster.getFileSystem().getClass().getName() + ", fs.default.name : " + props.getProperty("fs.default.name"));
+
+	    Map.Entry<Table, List<Partition>> metadata = EximUtil.readMetaData(fs, new Path(exportLocation, "_metadata"));
+	    Table table = metadata.getKey();
+	    List<Partition> partitions = metadata.getValue();
+
+	    List<HCatFieldSchema> columns = new ArrayList<HCatFieldSchema>();
+	    columns.add(HCatSchemaUtils.getHCatFieldSchema(new FieldSchema("emp_id",
+	        Constants.INT_TYPE_NAME, "")));
+	    columns.add(HCatSchemaUtils.getHCatFieldSchema(new FieldSchema("emp_name",
+	        Constants.STRING_TYPE_NAME, "")));
+	    columns.add(HCatSchemaUtils.getHCatFieldSchema(new FieldSchema("emp_dob",
+	        Constants.STRING_TYPE_NAME, "")));
+	    columns.add(HCatSchemaUtils.getHCatFieldSchema(new FieldSchema("emp_sex",
+	        Constants.STRING_TYPE_NAME, "")));
+
+
+	    assertEquals("default", table.getDbName());
+	    assertEquals(PARTITIONED_TABLE, table.getTableName());
+	    assertTrue(EximUtil.schemaCompare(table.getSd().getCols(),
+	        HCatUtil.getFieldSchemaList(columns)));
+	    assertEquals("org.apache.hcatalog.rcfile.RCFileInputDriver",
+	        table.getParameters().get(HCatConstants.HCAT_ISD_CLASS));
+	    assertEquals("org.apache.hcatalog.rcfile.RCFileOutputDriver",
+	        table.getParameters().get(HCatConstants.HCAT_OSD_CLASS));
+	    assertEquals("org.apache.hadoop.hive.ql.io.RCFileInputFormat",
+	        table.getSd().getInputFormat());
+	    assertEquals("org.apache.hadoop.hive.ql.io.RCFileOutputFormat",
+	        table.getSd().getOutputFormat());
+	    assertEquals("org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe",
+	        table.getSd().getSerdeInfo().getSerializationLib());
+	    assertEquals(2, table.getPartitionKeys().size());
+	    List<FieldSchema> partSchema = table.getPartitionKeys();
+	    assertEquals("emp_state", partSchema.get(0).getName());
+	    assertEquals("emp_country", partSchema.get(1).getName());
+
+	    assertEquals(1, partitions.size());
+	    Partition partition = partitions.get(0);
+	    assertEquals("tn", partition.getValues().get(0));
+	    assertEquals("in", partition.getValues().get(1));
+	    assertEquals("org.apache.hcatalog.rcfile.RCFileInputDriver",
+	        partition.getParameters().get(HCatConstants.HCAT_ISD_CLASS));
+	    assertEquals("org.apache.hcatalog.rcfile.RCFileOutputDriver",
+	        partition.getParameters().get(HCatConstants.HCAT_OSD_CLASS));
+	  }
+
   public void testStoreNonPartCompatSchemaTable() throws Exception {
     populateDataFile();
     PigServer server = new PigServer(ExecType.LOCAL, props);