moveTaskOutputs(fs, src, src, tblPath);
fs.delete(src, true);
return;
}
HiveMetaStoreClient client = null;
List<String> values = null;
boolean partitionAdded = false;
HowlTableInfo tableInfo = jobInfo.getTableInfo();
try {
client = HowlOutputFormat.createHiveClient(tableInfo.getServerUri(), conf);
StorerInfo storer = InitializeInput.extractStorerInfo(table.getSd(),table.getParameters());
Partition partition = new Partition();
partition.setDbName(tableInfo.getDatabaseName());
partition.setTableName(tableInfo.getTableName());
partition.setSd(new StorageDescriptor(tblSD));
partition.getSd().setLocation(jobInfo.getLocation());
updateTableSchema(client, table, jobInfo.getOutputSchema());
List<FieldSchema> fields = new ArrayList<FieldSchema>();
for(HowlFieldSchema fieldSchema : jobInfo.getOutputSchema().getFields()) {
fields.add(HowlSchemaUtils.getFieldSchema(fieldSchema));
}
partition.getSd().setCols(fields);
Map<String,String> partKVs = tableInfo.getPartitionValues();
//Get partition value list
partition.setValues(getPartitionValueList(table,partKVs));
Map<String, String> params = new HashMap<String, String>();
params.put(HowlConstants.HOWL_ISD_CLASS, storer.getInputSDClass());
params.put(HowlConstants.HOWL_OSD_CLASS, storer.getOutputSDClass());
//Copy table level howl.* keys to the partition
for(Map.Entry<Object, Object> entry : storer.getProperties().entrySet()) {
params.put(entry.getKey().toString(), entry.getValue().toString());
}
partition.setParameters(params);
// Sets permissions and group name on partition dirs.
FileStatus tblStat = fs.getFileStatus(tblPath);
String grpName = tblStat.getGroup();
FsPermission perms = tblStat.getPermission();
Path partPath = tblPath;
for(FieldSchema partKey : table.getPartitionKeys()){
partPath = constructPartialPartPath(partPath, partKey.getName().toLowerCase(), partKVs);
fs.setPermission(partPath, perms);
try{
fs.setOwner(partPath, null, grpName);
} catch(AccessControlException ace){
// log the messages before ignoring. Currently, logging is not built in Howl.
}
}
//Publish the new partition
client.add_partition(partition);
partitionAdded = true; //publish to metastore done
if( baseCommitter != null ) {
baseCommitter.cleanupJob(context);
}
// cancel the deleg. tokens that were acquired for this job now that
// we are done - we should cancel if the tokens were acquired by
// HowlOutputFormat and not if they were supplied by Oozie. In the latter
// case the HOWL_KEY_TOKEN_SIGNATURE property in the conf will not be set
String tokenStrForm = client.getTokenStrForm();
if(tokenStrForm != null && context.getConfiguration().get
(HowlOutputFormat.HOWL_KEY_TOKEN_SIGNATURE) != null) {
client.cancelDelegationToken(tokenStrForm);
}
} catch (Exception e) {
if( partitionAdded ) {
try {
//baseCommitter.cleanupJob failed, try to clean up the metastore
client.dropPartition(tableInfo.getDatabaseName(),
tableInfo.getTableName(), values);
} catch(Exception te) {
//Keep cause as the original exception
throw new HowlException(ErrorType.ERROR_PUBLISHING_PARTITION, e);
}
}
if( e instanceof HowlException ) {
throw (HowlException) e;
} else {
throw new HowlException(ErrorType.ERROR_PUBLISHING_PARTITION, e);
}
} finally {
if( client != null ) {
client.close();
}
}
}