Table table = client.getTable(outputInfo.getDatabaseName(), outputInfo.getTableName());
if (table.getPartitionKeysSize() == 0 ){
if ((outputInfo.getPartitionValues() != null) && (!outputInfo.getPartitionValues().isEmpty())){
// attempt made to save partition values in non-partitioned table - throw error.
throw new HCatException(ErrorType.ERROR_INVALID_PARTITION_VALUES,
"Partition values specified for non-partitioned table");
}
// non-partitioned table
outputInfo.setPartitionValues(new HashMap<String, String>());
} else {
// partitioned table, we expect partition values
// convert user specified map to have lower case key names
Map<String, String> valueMap = new HashMap<String, String>();
if (outputInfo.getPartitionValues() != null){
for(Map.Entry<String, String> entry : outputInfo.getPartitionValues().entrySet()) {
valueMap.put(entry.getKey().toLowerCase(), entry.getValue());
}
}
if (
(outputInfo.getPartitionValues() == null)
|| (outputInfo.getPartitionValues().size() < table.getPartitionKeysSize())
){
// dynamic partition usecase - partition values were null, or not all were specified
// need to figure out which keys are not specified.
List<String> dynamicPartitioningKeys = new ArrayList<String>();
boolean firstItem = true;
for (FieldSchema fs : table.getPartitionKeys()){
if (!valueMap.containsKey(fs.getName().toLowerCase())){
dynamicPartitioningKeys.add(fs.getName().toLowerCase());
}
}
if (valueMap.size() + dynamicPartitioningKeys.size() != table.getPartitionKeysSize()){
// If this isn't equal, then bogus key values have been inserted, error out.
throw new HCatException(ErrorType.ERROR_INVALID_PARTITION_VALUES,"Invalid partition keys specified");
}
outputInfo.setDynamicPartitioningKeys(dynamicPartitioningKeys);
String dynHash;
if ((dynHash = conf.get(HCatConstants.HCAT_DYNAMIC_PTN_JOBID)) == null){
dynHash = String.valueOf(Math.random());
// LOG.info("New dynHash : ["+dynHash+"]");
// }else{
// LOG.info("Old dynHash : ["+dynHash+"]");
}
conf.set(HCatConstants.HCAT_DYNAMIC_PTN_JOBID, dynHash);
}
outputInfo.setPartitionValues(valueMap);
}
//Handle duplicate publish
handleDuplicatePublish(job, outputInfo, client, table);
StorageDescriptor tblSD = table.getSd();
HCatSchema tableSchema = HCatUtil.extractSchemaFromStorageDescriptor(tblSD);
StorerInfo storerInfo = InitializeInput.extractStorerInfo(tblSD,table.getParameters());
List<String> partitionCols = new ArrayList<String>();
for(FieldSchema schema : table.getPartitionKeys()) {
partitionCols.add(schema.getName());
}
Class<? extends HCatOutputStorageDriver> driverClass =
(Class<? extends HCatOutputStorageDriver>) Class.forName(storerInfo.getOutputSDClass());
HCatOutputStorageDriver driver = driverClass.newInstance();
String tblLocation = tblSD.getLocation();
String location = driver.getOutputLocation(job,
tblLocation, partitionCols,
outputInfo.getPartitionValues(),conf.get(HCatConstants.HCAT_DYNAMIC_PTN_JOBID));
//Serialize the output info into the configuration
OutputJobInfo jobInfo = new OutputJobInfo(outputInfo,
tableSchema, tableSchema, storerInfo, location, table);
jobInfo.setHarRequested(harRequested);
jobInfo.setMaximumDynamicPartitions(maxDynamicPartitions);
conf.set(HCatConstants.HCAT_KEY_OUTPUT_INFO, HCatUtil.serialize(jobInfo));
Path tblPath = new Path(tblLocation);
/* Set the umask in conf such that files/dirs get created with table-dir
* permissions. Following three assumptions are made:
* 1. Actual files/dirs creation is done by RecordWriter of underlying
* output format. It is assumed that they use default permissions while creation.
* 2. Default Permissions = FsPermission.getDefault() = 777.
* 3. UMask is honored by underlying filesystem.
*/
FsPermission.setUMask(conf, FsPermission.getDefault().applyUMask(
tblPath.getFileSystem(conf).getFileStatus(tblPath).getPermission()));
if(UserGroupInformation.isSecurityEnabled()){
UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
// check if oozie has set up a hcat deleg. token - if so use it
TokenSelector<? extends TokenIdentifier> tokenSelector = new DelegationTokenSelector();
// TODO: will oozie use a "service" called "oozie" - then instead of
// new Text() do new Text("oozie") below - if this change is made also
// remember to do:
// job.getConfiguration().set(HCAT_KEY_TOKEN_SIGNATURE, "oozie");
// Also change code in HCatOutputCommitter.cleanupJob() to cancel the
// token only if token.service is not "oozie" - remove the condition of
// HCAT_KEY_TOKEN_SIGNATURE != null in that code.
Token<? extends TokenIdentifier> token = tokenSelector.selectToken(
new Text(), ugi.getTokens());
if(token != null) {
job.getCredentials().addToken(new Text(ugi.getUserName()),token);
} else {
// we did not get token set up by oozie, let's get them ourselves here.
// we essentially get a token per unique Output HCatTableInfo - this is
// done because through Pig, setOutput() method is called multiple times
// We want to only get the token once per unique output HCatTableInfo -
// we cannot just get one token since in multi-query case (> 1 store in 1 job)
// or the case when a single pig script results in > 1 jobs, the single
// token will get cancelled by the output committer and the subsequent
// stores will fail - by tying the token with the concatenation of
// dbname, tablename and partition keyvalues of the output
// TableInfo, we can have as many tokens as there are stores and the TokenSelector
// will correctly pick the right tokens which the committer will use and
// cancel.
String tokenSignature = getTokenSignature(outputInfo);
if(tokenMap.get(tokenSignature) == null) {
// get delegation tokens from hcat server and store them into the "job"
// These will be used in the HCatOutputCommitter to publish partitions to
// hcat
// when the JobTracker in Hadoop MapReduce starts supporting renewal of
// arbitrary tokens, the renewer should be the principal of the JobTracker
tokenMap.put(tokenSignature, HCatUtil.extractThriftToken(
client.getDelegationToken(ugi.getUserName()),
tokenSignature));
}
String jcTokenSignature = "jc."+tokenSignature;
if (harRequested){
if(tokenMap.get(jcTokenSignature) == null) {
tokenMap.put(jcTokenSignature,
HCatUtil.getJobTrackerDelegationToken(conf,ugi.getUserName()));
}
}
job.getCredentials().addToken(new Text(ugi.getUserName() + tokenSignature),
tokenMap.get(tokenSignature));
// this will be used by the outputcommitter to pass on to the metastore client
// which in turn will pass on to the TokenSelector so that it can select
// the right token.
job.getConfiguration().set(HCatConstants.HCAT_KEY_TOKEN_SIGNATURE, tokenSignature);
if (harRequested){
job.getCredentials().addToken(new Text(ugi.getUserName() + jcTokenSignature),
tokenMap.get(jcTokenSignature));
job.getConfiguration().set(
HCatConstants.HCAT_KEY_JOBCLIENT_TOKEN_SIGNATURE, jcTokenSignature);
job.getConfiguration().set(
HCatConstants.HCAT_KEY_JOBCLIENT_TOKEN_STRFORM,
tokenMap.get(jcTokenSignature).encodeToUrlString());
// LOG.info("Set hive dt["+tokenSignature+"]");
// LOG.info("Set jt dt["+jcTokenSignature+"]");
}
}
}
} catch(Exception e) {
if( e instanceof HCatException ) {
throw (HCatException) e;
} else {
throw new HCatException(ErrorType.ERROR_SET_OUTPUT, e);
}
} finally {
if( client != null ) {
client.close();
}