Index: src/test/org/apache/hcatalog/pig/TestHCatEximStorer.java
===================================================================
--- src/test/org/apache/hcatalog/pig/TestHCatEximStorer.java (revision 1182648)
+++ src/test/org/apache/hcatalog/pig/TestHCatEximStorer.java (working copy)
@@ -196,6 +196,64 @@
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
> metadata = EximUtil.readMetaData(fs, new Path(exportLocation, "_metadata"));
+ Table table = metadata.getKey();
+ List partitions = metadata.getValue();
+
+ List columns = new ArrayList();
+ 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 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);
Index: src/java/org/apache/hcatalog/pig/HCatEximStorer.java
===================================================================
--- src/java/org/apache/hcatalog/pig/HCatEximStorer.java (revision 1182648)
+++ src/java/org/apache/hcatalog/pig/HCatEximStorer.java (working copy)
@@ -107,7 +107,7 @@
List hcatFields = new ArrayList();
List partVals = new ArrayList();
- for (String key : partitions.keySet()) {
+ for (String key : partitionKeys) {
hcatFields.add(new HCatFieldSchema(key, HCatFieldSchema.Type.STRING, ""));
partVals.add(partitions.get(key));
}
Index: src/java/org/apache/hcatalog/pig/HCatBaseStorer.java
===================================================================
--- src/java/org/apache/hcatalog/pig/HCatBaseStorer.java (revision 1182648)
+++ src/java/org/apache/hcatalog/pig/HCatBaseStorer.java (working copy)
@@ -62,6 +62,7 @@
*
*/
protected static final String COMPUTED_OUTPUT_SCHEMA = "hcat.output.schema";
+ protected final List partitionKeys;
protected final Map partitions;
protected Schema pigSchema;
private RecordWriter, HCatRecord> writer;
@@ -71,13 +72,16 @@
public HCatBaseStorer(String partSpecs, String schema) throws Exception {
+ partitionKeys = new ArrayList();
partitions = new HashMap();
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);
}
Index: src/java/org/apache/hcatalog/mapreduce/HCatEximOutputFormat.java
===================================================================
--- src/java/org/apache/hcatalog/mapreduce/HCatEximOutputFormat.java (revision 1182648)
+++ src/java/org/apache/hcatalog/mapreduce/HCatEximOutputFormat.java (working copy)
@@ -28,6 +28,7 @@
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;
@@ -145,8 +146,9 @@
try {
String partname = null;
if ((partKeys != null) && !partKeys.isEmpty()) {
- table.setPartitionKeys(HCatSchemaUtils.getFieldSchemas(partKeys));
- partname = Warehouse.makePartPath(partSpec);
+ List partSchema = HCatSchemaUtils.getFieldSchemas(partKeys);
+ table.setPartitionKeys(partSchema);
+ partname = Warehouse.makePartName(partSchema, partitionValues);
} else {
partname = "data";
}