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 tr...@apache.org on 2012/10/18 00:51:15 UTC
svn commit: r1399498 - in /incubator/hcatalog/trunk: ./
src/java/org/apache/hcatalog/common/ src/java/org/apache/hcatalog/mapreduce/
Author: travis
Date: Thu Oct 18 00:51:14 2012
New Revision: 1399498
URL: http://svn.apache.org/viewvc?rev=1399498&view=rev
Log:
HCATALOG-516 HCOF should use less generic parameters
Modified:
incubator/hcatalog/trunk/CHANGES.txt
incubator/hcatalog/trunk/src/java/org/apache/hcatalog/common/HCatUtil.java
incubator/hcatalog/trunk/src/java/org/apache/hcatalog/mapreduce/HCatBaseInputFormat.java
incubator/hcatalog/trunk/src/java/org/apache/hcatalog/mapreduce/HCatBaseOutputFormat.java
incubator/hcatalog/trunk/src/java/org/apache/hcatalog/mapreduce/HCatInputFormat.java
incubator/hcatalog/trunk/src/java/org/apache/hcatalog/mapreduce/HCatOutputFormat.java
incubator/hcatalog/trunk/src/java/org/apache/hcatalog/mapreduce/InitializeInput.java
incubator/hcatalog/trunk/src/java/org/apache/hcatalog/mapreduce/Security.java
Modified: incubator/hcatalog/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/incubator/hcatalog/trunk/CHANGES.txt?rev=1399498&r1=1399497&r2=1399498&view=diff
==============================================================================
--- incubator/hcatalog/trunk/CHANGES.txt (original)
+++ incubator/hcatalog/trunk/CHANGES.txt Thu Oct 18 00:51:14 2012
@@ -44,6 +44,8 @@ Trunk (unreleased changes)
HCAT-427 Document storage-based authorization (lefty via gates)
IMPROVEMENTS
+ HCAT-516 HCOF should use less generic parameters (nitay via traviscrawford)
+
HCAT-525 Remove dead classes from HCatalog Source Tree (amalakar via traviscrawford)
HCAT-521 Ignore .reviewboardrc in git (nitay via traviscrawford)
Modified: incubator/hcatalog/trunk/src/java/org/apache/hcatalog/common/HCatUtil.java
URL: http://svn.apache.org/viewvc/incubator/hcatalog/trunk/src/java/org/apache/hcatalog/common/HCatUtil.java?rev=1399498&r1=1399497&r2=1399498&view=diff
==============================================================================
--- incubator/hcatalog/trunk/src/java/org/apache/hcatalog/common/HCatUtil.java (original)
+++ incubator/hcatalog/trunk/src/java/org/apache/hcatalog/common/HCatUtil.java Thu Oct 18 00:51:14 2012
@@ -31,6 +31,8 @@ import java.util.List;
import java.util.Map;
import java.util.Properties;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.permission.FsAction;
import org.apache.hadoop.hive.common.JavaUtils;
@@ -467,10 +469,11 @@ public class HCatUtil {
return jobProperties;
}
-
+ @InterfaceAudience.Private
+ @InterfaceStability.Evolving
public static void
configureOutputStorageHandler(HCatStorageHandler storageHandler,
- JobContext context,
+ Configuration conf,
OutputJobInfo outputJobInfo) {
//TODO replace IgnoreKeyTextOutputFormat with a
//HiveOutputFormatWrapper in StorageHandler
@@ -480,7 +483,7 @@ public class HCatUtil {
outputJobInfo.getTableInfo().getStorerInfo().getProperties());
if (tableDesc.getJobProperties() == null)
tableDesc.setJobProperties(new HashMap<String, String>());
- for (Map.Entry<String, String> el : context.getConfiguration()) {
+ for (Map.Entry<String, String> el : conf) {
tableDesc.getJobProperties().put(el.getKey(), el.getValue());
}
@@ -494,7 +497,7 @@ public class HCatUtil {
jobProperties);
for (Map.Entry<String, String> el : jobProperties.entrySet()) {
- context.getConfiguration().set(el.getKey(), el.getValue());
+ conf.set(el.getKey(), el.getValue());
}
} catch (IOException e) {
throw new IllegalStateException(
Modified: incubator/hcatalog/trunk/src/java/org/apache/hcatalog/mapreduce/HCatBaseInputFormat.java
URL: http://svn.apache.org/viewvc/incubator/hcatalog/trunk/src/java/org/apache/hcatalog/mapreduce/HCatBaseInputFormat.java?rev=1399498&r1=1399497&r2=1399498&view=diff
==============================================================================
--- incubator/hcatalog/trunk/src/java/org/apache/hcatalog/mapreduce/HCatBaseInputFormat.java (original)
+++ incubator/hcatalog/trunk/src/java/org/apache/hcatalog/mapreduce/HCatBaseInputFormat.java Thu Oct 18 00:51:14 2012
@@ -58,12 +58,11 @@ public abstract class HCatBaseInputForma
private Class<? extends InputFormat> inputFileFormatClass;
// TODO needs to go in InitializeInput? as part of InputJobInfo
- public static HCatSchema getOutputSchema(JobContext context)
+ private static HCatSchema getOutputSchema(Configuration conf)
throws IOException {
- String os = context.getConfiguration().get(
- HCatConstants.HCAT_KEY_OUTPUT_SCHEMA);
+ String os = conf.get(HCatConstants.HCAT_KEY_OUTPUT_SCHEMA);
if (os == null) {
- return getTableSchema(context);
+ return getTableSchema(conf);
} else {
return (HCatSchema) HCatUtil.deserialize(os);
}
@@ -98,12 +97,13 @@ public abstract class HCatBaseInputForma
@Override
public List<InputSplit> getSplits(JobContext jobContext)
throws IOException, InterruptedException {
+ Configuration conf = jobContext.getConfiguration();
//Get the job info from the configuration,
//throws exception if not initialized
InputJobInfo inputJobInfo;
try {
- inputJobInfo = getJobInfo(jobContext);
+ inputJobInfo = getJobInfo(conf);
} catch (Exception e) {
throw new IOException(e);
}
@@ -117,7 +117,6 @@ public abstract class HCatBaseInputForma
HCatStorageHandler storageHandler;
JobConf jobConf;
- Configuration conf = jobContext.getConfiguration();
//For each matching partition, call getSplits on the underlying InputFormat
for (PartInfo partitionInfo : partitionInfoList) {
jobConf = HCatUtil.getJobConfFromContext(jobContext);
@@ -183,16 +182,17 @@ public abstract class HCatBaseInputForma
HCatSplit hcatSplit = InternalUtil.castToHCatSplit(split);
PartInfo partitionInfo = hcatSplit.getPartitionInfo();
JobContext jobContext = taskContext;
+ Configuration conf = jobContext.getConfiguration();
HCatStorageHandler storageHandler = HCatUtil.getStorageHandler(
- jobContext.getConfiguration(), partitionInfo);
+ conf, partitionInfo);
JobConf jobConf = HCatUtil.getJobConfFromContext(jobContext);
Map<String, String> jobProperties = partitionInfo.getJobProperties();
HCatUtil.copyJobPropertiesToJobConf(jobProperties, jobConf);
Map<String, String> valuesNotInDataCols = getColValsNotInDataColumns(
- getOutputSchema(jobContext), partitionInfo
+ getOutputSchema(conf), partitionInfo
);
return new HCatRecordReader(storageHandler, valuesNotInDataCols);
@@ -222,17 +222,27 @@ public abstract class HCatBaseInputForma
}
/**
+ * @see org.apache.hcatalog.mapreduce.HCatBaseInputFormat#getTableSchema(org.apache.hadoop.conf.Configuration)
+ * @deprecated Use {@link #getTableSchema(org.apache.hadoop.conf.Configuration)}
+ */
+ public static HCatSchema getTableSchema(JobContext context)
+ throws IOException {
+ return getTableSchema(context.getConfiguration());
+ }
+
+
+ /**
* Gets the HCatTable schema for the table specified in the HCatInputFormat.setInput call
* on the specified job context. This information is available only after HCatInputFormat.setInput
* has been called for a JobContext.
- * @param context the context
+ * @param conf the Configuration object
* @return the table schema
* @throws IOException if HCatInputFormat.setInput has not been called
* for the current context
*/
- public static HCatSchema getTableSchema(JobContext context)
+ public static HCatSchema getTableSchema(Configuration conf)
throws IOException {
- InputJobInfo inputJobInfo = getJobInfo(context);
+ InputJobInfo inputJobInfo = getJobInfo(conf);
HCatSchema allCols = new HCatSchema(new LinkedList<HCatFieldSchema>());
for (HCatFieldSchema field :
inputJobInfo.getTableInfo().getDataColumns().getFields())
@@ -247,13 +257,13 @@ public abstract class HCatBaseInputForma
* Gets the InputJobInfo object by reading the Configuration and deserializing
* the string. If InputJobInfo is not present in the configuration, throws an
* exception since that means HCatInputFormat.setInput has not been called.
- * @param jobContext the job context
+ * @param conf the Configuration object
* @return the InputJobInfo object
* @throws IOException the exception
*/
- private static InputJobInfo getJobInfo(JobContext jobContext)
+ private static InputJobInfo getJobInfo(Configuration conf)
throws IOException {
- String jobString = jobContext.getConfiguration().get(
+ String jobString = conf.get(
HCatConstants.HCAT_KEY_JOB_INFO);
if (jobString == null) {
throw new IOException("job information not found in JobContext."
Modified: incubator/hcatalog/trunk/src/java/org/apache/hcatalog/mapreduce/HCatBaseOutputFormat.java
URL: http://svn.apache.org/viewvc/incubator/hcatalog/trunk/src/java/org/apache/hcatalog/mapreduce/HCatBaseOutputFormat.java?rev=1399498&r1=1399497&r2=1399498&view=diff
==============================================================================
--- incubator/hcatalog/trunk/src/java/org/apache/hcatalog/mapreduce/HCatBaseOutputFormat.java (original)
+++ incubator/hcatalog/trunk/src/java/org/apache/hcatalog/mapreduce/HCatBaseOutputFormat.java Thu Oct 18 00:51:14 2012
@@ -23,6 +23,7 @@ import java.util.ArrayList;
import java.util.List;
import java.util.Map;
+import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hive.ql.metadata.Table;
import org.apache.hadoop.io.WritableComparable;
import org.apache.hadoop.mapreduce.JobContext;
@@ -40,14 +41,22 @@ public abstract class HCatBaseOutputForm
// static final private Log LOG = LogFactory.getLog(HCatBaseOutputFormat.class);
/**
+ * @see org.apache.hcatalog.mapreduce.HCatBaseOutputFormat#getTableSchema(org.apache.hadoop.conf.Configuration)
+ * @deprecated Use {@link #getTableSchema(org.apache.hadoop.conf.Configuration)}
+ */
+ public static HCatSchema getTableSchema(JobContext context) throws IOException {
+ return getTableSchema(context.getConfiguration());
+ }
+
+ /**
* Gets the table schema for the table specified in the HCatOutputFormat.setOutput call
* on the specified job context.
- * @param context the context
+ * @param conf the Configuration object
* @return the table schema
- * @throws IOException if HCatOutputFromat.setOutput has not been called for the passed context
+ * @throws IOException if HCatOutputFormat.setOutput has not been called for the passed context
*/
- public static HCatSchema getTableSchema(JobContext context) throws IOException {
- OutputJobInfo jobInfo = getJobInfo(context);
+ public static HCatSchema getTableSchema(Configuration conf) throws IOException {
+ OutputJobInfo jobInfo = getJobInfo(conf);
return jobInfo.getTableInfo().getDataColumns();
}
@@ -77,15 +86,23 @@ public abstract class HCatBaseOutputForm
}
/**
+ * @see org.apache.hcatalog.mapreduce.HCatBaseOutputFormat#getJobInfo(org.apache.hadoop.conf.Configuration)
+ * @deprecated use {@link #getJobInfo(org.apache.hadoop.conf.Configuration)}
+ */
+ public static OutputJobInfo getJobInfo(JobContext jobContext) throws IOException {
+ return getJobInfo(jobContext.getConfiguration());
+ }
+
+ /**
* Gets the HCatOuputJobInfo object by reading the Configuration and deserializing
* the string. If InputJobInfo is not present in the configuration, throws an
* exception since that means HCatOutputFormat.setOutput has not been called.
- * @param jobContext the job context
+ * @param conf the job Configuration object
* @return the OutputJobInfo object
* @throws IOException the IO exception
*/
- public static OutputJobInfo getJobInfo(JobContext jobContext) throws IOException {
- String jobString = jobContext.getConfiguration().get(HCatConstants.HCAT_KEY_OUTPUT_INFO);
+ public static OutputJobInfo getJobInfo(Configuration conf) throws IOException {
+ String jobString = conf.get(HCatConstants.HCAT_KEY_OUTPUT_INFO);
if (jobString == null) {
throw new HCatException(ErrorType.ERROR_NOT_INITIALIZED);
}
@@ -113,9 +130,10 @@ public abstract class HCatBaseOutputForm
@SuppressWarnings("unchecked")
static void configureOutputStorageHandler(
JobContext jobContext, List<String> dynamicPartVals) throws IOException {
+ Configuration conf = jobContext.getConfiguration();
try {
- OutputJobInfo jobInfo = (OutputJobInfo) HCatUtil.deserialize(jobContext.getConfiguration().get(HCatConstants.HCAT_KEY_OUTPUT_INFO));
- HCatStorageHandler storageHandler = HCatUtil.getStorageHandler(jobContext.getConfiguration(), jobInfo.getTableInfo().getStorerInfo());
+ OutputJobInfo jobInfo = (OutputJobInfo) HCatUtil.deserialize(conf.get(HCatConstants.HCAT_KEY_OUTPUT_INFO));
+ HCatStorageHandler storageHandler = HCatUtil.getStorageHandler(conf, jobInfo.getTableInfo().getStorerInfo());
Map<String, String> partitionValues = jobInfo.getPartitionValues();
String location = jobInfo.getLocation();
@@ -143,7 +161,7 @@ public abstract class HCatBaseOutputForm
jobInfo.setPartitionValues(partitionValues);
}
- HCatUtil.configureOutputStorageHandler(storageHandler, jobContext, jobInfo);
+ HCatUtil.configureOutputStorageHandler(storageHandler, conf, jobInfo);
} catch (Exception e) {
if (e instanceof HCatException) {
throw (HCatException) e;
Modified: incubator/hcatalog/trunk/src/java/org/apache/hcatalog/mapreduce/HCatInputFormat.java
URL: http://svn.apache.org/viewvc/incubator/hcatalog/trunk/src/java/org/apache/hcatalog/mapreduce/HCatInputFormat.java?rev=1399498&r1=1399497&r2=1399498&view=diff
==============================================================================
--- incubator/hcatalog/trunk/src/java/org/apache/hcatalog/mapreduce/HCatInputFormat.java (original)
+++ incubator/hcatalog/trunk/src/java/org/apache/hcatalog/mapreduce/HCatInputFormat.java Thu Oct 18 00:51:14 2012
@@ -20,24 +20,33 @@ package org.apache.hcatalog.mapreduce;
import java.io.IOException;
+import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.mapreduce.Job;
/** The InputFormat to use to read data from HCatalog. */
public class HCatInputFormat extends HCatBaseInputFormat {
/**
+ * @see org.apache.hcatalog.mapreduce.HCatInputFormat#setInput(org.apache.hadoop.conf.Configuration, InputJobInfo)
+ */
+ public static void setInput(Job job,
+ InputJobInfo inputJobInfo) throws IOException {
+ setInput(job.getConfiguration(), inputJobInfo);
+ }
+
+ /**
* Set the input information to use for the job. This queries the metadata server
* with the specified partition predicates, gets the matching partitions, and
* puts the information in the conf object. The inputInfo object is updated
* with information needed in the client context.
- * @param job the job object
+ * @param conf the job Configuration object
* @param inputJobInfo the input information about the table to read
* @throws IOException the exception in communicating with the metadata server
*/
- public static void setInput(Job job,
+ public static void setInput(Configuration conf,
InputJobInfo inputJobInfo) throws IOException {
try {
- InitializeInput.setInput(job, inputJobInfo);
+ InitializeInput.setInput(conf, inputJobInfo);
} catch (Exception e) {
throw new IOException(e);
}
Modified: incubator/hcatalog/trunk/src/java/org/apache/hcatalog/mapreduce/HCatOutputFormat.java
URL: http://svn.apache.org/viewvc/incubator/hcatalog/trunk/src/java/org/apache/hcatalog/mapreduce/HCatOutputFormat.java?rev=1399498&r1=1399497&r2=1399498&view=diff
==============================================================================
--- incubator/hcatalog/trunk/src/java/org/apache/hcatalog/mapreduce/HCatOutputFormat.java (original)
+++ incubator/hcatalog/trunk/src/java/org/apache/hcatalog/mapreduce/HCatOutputFormat.java Thu Oct 18 00:51:14 2012
@@ -38,6 +38,7 @@ import org.apache.hadoop.mapreduce.Job;
import org.apache.hadoop.mapreduce.OutputCommitter;
import org.apache.hadoop.mapreduce.RecordWriter;
import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.security.Credentials;
import org.apache.hcatalog.common.ErrorType;
import org.apache.hcatalog.common.HCatConstants;
import org.apache.hcatalog.common.HCatException;
@@ -57,20 +58,27 @@ public class HCatOutputFormat extends HC
private static boolean harRequested;
/**
+ * @see org.apache.hcatalog.mapreduce.HCatOutputFormat#setOutput(org.apache.hadoop.conf.Configuration, OutputJobInfo)
+ */
+ public static void setOutput(Job job, OutputJobInfo outputJobInfo) throws IOException {
+ setOutput(job.getConfiguration(), job.getCredentials(), outputJobInfo);
+ }
+
+ /**
* Set the information about the output to write for the job. This queries the metadata server
- * to find the StorageHandler to use for the table. It throws an error if the
+ * to find the StorageHandler to use for the table. It throws an error if the
* partition is already published.
* @param job the job object
* @param outputJobInfo the table output information for the job
* @throws IOException the exception in communicating with the metadata server
*/
@SuppressWarnings("unchecked")
- public static void setOutput(Job job, OutputJobInfo outputJobInfo) throws IOException {
+ public static void setOutput(Configuration conf, Credentials credentials,
+ OutputJobInfo outputJobInfo) throws IOException {
HiveMetaStoreClient client = null;
try {
- Configuration conf = job.getConfiguration();
HiveConf hiveConf = HCatUtil.getHiveConf(conf);
client = HCatUtil.getHiveClient(hiveConf);
Table table = HCatUtil.getTable(client, outputJobInfo.getDatabaseName(),
@@ -158,7 +166,7 @@ public class HCatOutputFormat extends HC
partitionCols.add(schema.getName());
}
- HCatStorageHandler storageHandler = HCatUtil.getStorageHandler(job.getConfiguration(), storerInfo);
+ HCatStorageHandler storageHandler = HCatUtil.getStorageHandler(conf, storerInfo);
//Serialize the output info into the configuration
outputJobInfo.setTableInfo(HCatTableInfo.valueOf(table.getTTable()));
@@ -168,7 +176,7 @@ public class HCatOutputFormat extends HC
maxDynamicPartitions = getMaxDynamicPartitions(hiveConf);
outputJobInfo.setMaximumDynamicPartitions(maxDynamicPartitions);
- HCatUtil.configureOutputStorageHandler(storageHandler, job, outputJobInfo);
+ HCatUtil.configureOutputStorageHandler(storageHandler, conf, outputJobInfo);
Path tblPath = new Path(table.getTTable().getSd().getLocation());
@@ -184,7 +192,7 @@ public class HCatOutputFormat extends HC
tblPath.getFileSystem(conf).getFileStatus(tblPath).getPermission()));
if (Security.getInstance().isSecurityEnabled()) {
- Security.getInstance().handleSecurity(job, outputJobInfo, client, conf, harRequested);
+ Security.getInstance().handleSecurity(credentials, outputJobInfo, client, conf, harRequested);
}
} catch (Exception e) {
if (e instanceof HCatException) {
@@ -198,18 +206,24 @@ public class HCatOutputFormat extends HC
}
/**
+ * @see org.apache.hcatalog.mapreduce.HCatOutputFormat#setSchema(org.apache.hadoop.conf.Configuration, org.apache.hcatalog.data.schema.HCatSchema)
+ */
+ public static void setSchema(final Job job, final HCatSchema schema) throws IOException {
+ setSchema(job.getConfiguration(), schema);
+ }
+
+ /**
* Set the schema for the data being written out to the partition. The
* table schema is used by default for the partition if this is not called.
- * @param job the job object
+ * @param conf the job Configuration object
* @param schema the schema for the data
* @throws IOException
*/
- public static void setSchema(final Job job, final HCatSchema schema) throws IOException {
-
- OutputJobInfo jobInfo = getJobInfo(job);
+ public static void setSchema(final Configuration conf, final HCatSchema schema) throws IOException {
+ OutputJobInfo jobInfo = getJobInfo(conf);
Map<String, String> partMap = jobInfo.getPartitionValues();
setPartDetails(jobInfo, schema, partMap);
- job.getConfiguration().set(HCatConstants.HCAT_KEY_OUTPUT_INFO, HCatUtil.serialize(jobInfo));
+ conf.set(HCatConstants.HCAT_KEY_OUTPUT_INFO, HCatUtil.serialize(jobInfo));
}
/**
Modified: incubator/hcatalog/trunk/src/java/org/apache/hcatalog/mapreduce/InitializeInput.java
URL: http://svn.apache.org/viewvc/incubator/hcatalog/trunk/src/java/org/apache/hcatalog/mapreduce/InitializeInput.java?rev=1399498&r1=1399497&r2=1399498&view=diff
==============================================================================
--- incubator/hcatalog/trunk/src/java/org/apache/hcatalog/mapreduce/InitializeInput.java (original)
+++ incubator/hcatalog/trunk/src/java/org/apache/hcatalog/mapreduce/InitializeInput.java Thu Oct 18 00:51:14 2012
@@ -50,6 +50,13 @@ public class InitializeInput {
private static final Logger LOG = LoggerFactory.getLogger(InitializeInput.class);
/**
+ * @see org.apache.hcatalog.mapreduce.InitializeInput#setInput(org.apache.hadoop.conf.Configuration, InputJobInfo)
+ */
+ public static void setInput(Job job, InputJobInfo theirInputJobInfo) throws Exception {
+ setInput(job.getConfiguration(), theirInputJobInfo);
+ }
+
+ /**
* Set the input to use for the Job. This queries the metadata server with the specified
* partition predicates, gets the matching partitions, and puts the information in the job
* configuration object.
@@ -63,32 +70,32 @@ public class InitializeInput {
* job.getConfiguration().get(HCatConstants.HCAT_KEY_JOB_INFO));
* {code}
*
- * @param job the job object
+ * @param conf the job Configuration object
* @param theirInputJobInfo information on the Input to read
* @throws Exception
*/
- public static void setInput(Job job, InputJobInfo theirInputJobInfo) throws Exception {
+ public static void setInput(Configuration conf,
+ InputJobInfo theirInputJobInfo) throws Exception {
InputJobInfo inputJobInfo = InputJobInfo.create(
theirInputJobInfo.getDatabaseName(),
theirInputJobInfo.getTableName(),
theirInputJobInfo.getFilter());
inputJobInfo.getProperties().putAll(theirInputJobInfo.getProperties());
- job.getConfiguration().set(
+ conf.set(
HCatConstants.HCAT_KEY_JOB_INFO,
- HCatUtil.serialize(getInputJobInfo(job, inputJobInfo, null)));
+ HCatUtil.serialize(getInputJobInfo(conf, inputJobInfo, null)));
}
/**
* Returns the given InputJobInfo after populating with data queried from the metadata service.
*/
private static InputJobInfo getInputJobInfo(
- Job job, InputJobInfo inputJobInfo, String locationFilter) throws Exception {
-
+ Configuration conf, InputJobInfo inputJobInfo, String locationFilter) throws Exception {
HiveMetaStoreClient client = null;
HiveConf hiveConf = null;
try {
- if (job != null) {
- hiveConf = HCatUtil.getHiveConf(job.getConfiguration());
+ if (conf != null) {
+ hiveConf = HCatUtil.getHiveConf(conf);
} else {
hiveConf = new HiveConf(HCatInputFormat.class);
}
@@ -117,7 +124,7 @@ public class InitializeInput {
HCatSchema schema = HCatUtil.extractSchema(
new org.apache.hadoop.hive.ql.metadata.Partition(table, ptn));
PartInfo partInfo = extractPartInfo(schema, ptn.getSd(),
- ptn.getParameters(), job.getConfiguration(), inputJobInfo);
+ ptn.getParameters(), conf, inputJobInfo);
partInfo.setPartitionValues(InternalUtil.createPtnKeyValueMap(table, ptn));
partInfoList.add(partInfo);
}
@@ -126,7 +133,7 @@ public class InitializeInput {
//Non partitioned table
HCatSchema schema = HCatUtil.extractSchema(table);
PartInfo partInfo = extractPartInfo(schema, table.getTTable().getSd(),
- table.getParameters(), job.getConfiguration(), inputJobInfo);
+ table.getParameters(), conf, inputJobInfo);
partInfo.setPartitionValues(new HashMap<String, String>());
partInfoList.add(partInfo);
}
Modified: incubator/hcatalog/trunk/src/java/org/apache/hcatalog/mapreduce/Security.java
URL: http://svn.apache.org/viewvc/incubator/hcatalog/trunk/src/java/org/apache/hcatalog/mapreduce/Security.java?rev=1399498&r1=1399497&r2=1399498&view=diff
==============================================================================
--- incubator/hcatalog/trunk/src/java/org/apache/hcatalog/mapreduce/Security.java (original)
+++ incubator/hcatalog/trunk/src/java/org/apache/hcatalog/mapreduce/Security.java Thu Oct 18 00:51:14 2012
@@ -31,6 +31,7 @@ import org.apache.hadoop.hive.thrift.Del
import org.apache.hadoop.io.Text;
import org.apache.hadoop.mapreduce.Job;
import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.security.Credentials;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.security.token.TokenIdentifier;
@@ -99,7 +100,7 @@ final class Security {
}
void handleSecurity(
- Job job,
+ Credentials credentials,
OutputJobInfo outputJobInfo,
HiveMetaStoreClient client,
Configuration conf,
@@ -144,20 +145,32 @@ final class Security {
if (jtToken == null) {
//we don't need to cancel this token as the TokenRenewer for JT tokens
//takes care of cancelling them
- job.getCredentials().addToken(new Text("hcat jt token"),
- HCatUtil.getJobTrackerDelegationToken(conf, ugi.getUserName()));
+ credentials.addToken(
+ new Text("hcat jt token"),
+ HCatUtil.getJobTrackerDelegationToken(conf, ugi.getUserName())
+ );
}
}
- job.getCredentials().addToken(new Text(ugi.getUserName() + "_" + tokenSignature), hiveToken);
+ credentials.addToken(new Text(ugi.getUserName() + "_" + tokenSignature), hiveToken);
// 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);
+ conf.set(HCatConstants.HCAT_KEY_TOKEN_SIGNATURE, tokenSignature);
}
}
}
+ void handleSecurity(
+ Job job,
+ OutputJobInfo outputJobInfo,
+ HiveMetaStoreClient client,
+ Configuration conf,
+ boolean harRequested)
+ throws IOException, MetaException, TException, Exception {
+ handleSecurity(job.getCredentials(), outputJobInfo, client, conf, harRequested);
+ }
+
// we should cancel hcat token if it was acquired by hcat
// and not if it was supplied (ie Oozie). In the latter
// case the HCAT_KEY_TOKEN_SIGNATURE property in the conf will not be set