commit 703f8b349bfff493e7449c27159f58a074164f26 Author: Mithun RK Date: Fri Sep 29 15:37:25 2017 -0700 HIVE-17656: Hive settings are not passed to Orc/Avro SerDes, when used from HCatalog diff --git a/hcatalog/core/src/main/java/org/apache/hive/hcatalog/common/HCatUtil.java b/hcatalog/core/src/main/java/org/apache/hive/hcatalog/common/HCatUtil.java index 81804cfb2e..9ebe11da29 100644 --- a/hcatalog/core/src/main/java/org/apache/hive/hcatalog/common/HCatUtil.java +++ b/hcatalog/core/src/main/java/org/apache/hive/hcatalog/common/HCatUtil.java @@ -382,6 +382,23 @@ public static HiveStorageHandler getStorageHandler(Configuration conf, StorerInf storerInfo.getOfClass()); } + /** + * Create an instance of a storage handler defined in storerInfo. If one cannot be found + * then FosterStorageHandler is used to encapsulate the InputFormat, OutputFormat and SerDe. + * This StorageHandler assumes the other supplied storage artifacts are for a file-based storage system. + * @param hiveConf HiveConf used to configure the Configurable StorageHandler + * @param storerInfo StorerInfo to definining the StorageHandler and InputFormat, OutputFormat and SerDe + * @return storageHandler instance + * @throws IOException + */ + public static HiveStorageHandler getStorageHandler(HiveConf hiveConf, StorerInfo storerInfo) throws IOException { + return getStorageHandler(hiveConf, + storerInfo.getStorageHandlerClass(), + storerInfo.getSerdeClass(), + storerInfo.getIfClass(), + storerInfo.getOfClass()); + } + public static HiveStorageHandler getStorageHandler(Configuration conf, PartInfo partitionInfo) throws IOException { return HCatUtil.getStorageHandler( conf, @@ -391,11 +408,20 @@ public static HiveStorageHandler getStorageHandler(Configuration conf, PartInfo partitionInfo.getOutputFormatClassName()); } + public static HiveStorageHandler getStorageHandler(HiveConf hiveConf, PartInfo partitionInfo) throws IOException { + return HCatUtil.getStorageHandler( + hiveConf, + partitionInfo.getStorageHandlerClassName(), + partitionInfo.getSerdeClassName(), + partitionInfo.getInputFormatClassName(), + partitionInfo.getOutputFormatClassName()); + } + /** * Create an instance of a storage handler. If storageHandler == null, * then surrrogate StorageHandler is used to encapsulate the InputFormat, OutputFormat and SerDe. * This StorageHandler assumes the other supplied storage artifacts are for a file-based storage system. - * @param conf job's configuration will be used to configure the Configurable StorageHandler + * @param hiveConf HiveConf that will be used to configure the Configurable StorageHandler * @param storageHandler fully qualified class name of the desired StorageHandle instance * @param serDe fully qualified class name of the desired SerDe instance * @param inputFormat fully qualified class name of the desired InputFormat instance @@ -403,7 +429,7 @@ public static HiveStorageHandler getStorageHandler(Configuration conf, PartInfo * @return storageHandler instance * @throws IOException */ - public static HiveStorageHandler getStorageHandler(Configuration conf, + public static HiveStorageHandler getStorageHandler(HiveConf hiveConf, String storageHandler, String serDe, String inputFormat, @@ -414,7 +440,7 @@ public static HiveStorageHandler getStorageHandler(Configuration conf, try { FosterStorageHandler fosterStorageHandler = new FosterStorageHandler(inputFormat, outputFormat, serDe); - fosterStorageHandler.setConf(conf); + fosterStorageHandler.setConf(hiveConf); return fosterStorageHandler; } catch (ClassNotFoundException e) { throw new IOException("Failed to load " @@ -427,13 +453,34 @@ public static HiveStorageHandler getStorageHandler(Configuration conf, (Class) Class .forName(storageHandler, true, Utilities.getSessionSpecifiedClassLoader()); return (HiveStorageHandler) ReflectionUtils.newInstance( - handlerClass, conf); + handlerClass, hiveConf); } catch (ClassNotFoundException e) { throw new IOException("Error in loading storage handler." + e.getMessage(), e); } } + /** + * Create an instance of a storage handler. If storageHandler == null, + * then surrogate StorageHandler is used to encapsulate the InputFormat, OutputFormat and SerDe. + * This StorageHandler assumes the other supplied storage artifacts are for a file-based storage system. + * @param conf job's configuration will be used to configure the Configurable StorageHandler + * @param storageHandler fully qualified class name of the desired StorageHandle instance + * @param serDe fully qualified class name of the desired SerDe instance + * @param inputFormat fully qualified class name of the desired InputFormat instance + * @param outputFormat fully qualified class name of the desired outputFormat instance + * @return storageHandler instance + * @throws IOException + */ + public static HiveStorageHandler getStorageHandler(Configuration conf, + String storageHandler, + String serDe, + String inputFormat, + String outputFormat) + throws IOException { + return getStorageHandler(getHiveConf(conf), storageHandler, serDe, inputFormat, outputFormat); + } + public static Pair getDbAndTableName(String tableName) throws IOException { String[] dbTableNametokens = tableName.split("\\."); if (dbTableNametokens.length == 1) { diff --git a/hcatalog/core/src/main/java/org/apache/hive/hcatalog/mapreduce/HCatBaseInputFormat.java b/hcatalog/core/src/main/java/org/apache/hive/hcatalog/mapreduce/HCatBaseInputFormat.java index ec1e1ca1bb..3574987081 100644 --- a/hcatalog/core/src/main/java/org/apache/hive/hcatalog/mapreduce/HCatBaseInputFormat.java +++ b/hcatalog/core/src/main/java/org/apache/hive/hcatalog/mapreduce/HCatBaseInputFormat.java @@ -29,6 +29,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.ql.metadata.HiveStorageHandler; import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorConverters; @@ -127,6 +128,7 @@ public static void setOutputSchema(Job job, HCatSchema hcatSchema) HiveStorageHandler storageHandler; Map hiveProps = null; + HiveConf hiveConf = HCatUtil.getHiveConf(jobContext.getConfiguration()); //For each matching partition, call getSplits on the underlying InputFormat for (PartInfo partitionInfo : partitionInfoList) { JobConf jobConf = HCatUtil.getJobConfFromContext(jobContext); @@ -142,8 +144,7 @@ public static void setOutputSchema(Job job, HCatSchema hcatSchema) HCatUtil.copyJobPropertiesToJobConf(hiveProps, jobConf); HCatUtil.copyJobPropertiesToJobConf(jobProperties, jobConf); - storageHandler = HCatUtil.getStorageHandler( - jobConf, partitionInfo); + storageHandler = HCatUtil.getStorageHandler(hiveConf, partitionInfo); //Get the input format Class inputFormatClass = storageHandler.getInputFormatClass(); diff --git a/hcatalog/core/src/main/java/org/apache/hive/hcatalog/mapreduce/InitializeInput.java b/hcatalog/core/src/main/java/org/apache/hive/hcatalog/mapreduce/InitializeInput.java index 2f07be1ae5..615bcd7148 100644 --- a/hcatalog/core/src/main/java/org/apache/hive/hcatalog/mapreduce/InitializeInput.java +++ b/hcatalog/core/src/main/java/org/apache/hive/hcatalog/mapreduce/InitializeInput.java @@ -126,7 +126,7 @@ private static InputJobInfo getInputJobInfo( HCatSchema schema = HCatUtil.extractSchema( new org.apache.hadoop.hive.ql.metadata.Partition(table, ptn)); PartInfo partInfo = extractPartInfo(schema, ptn.getSd(), - ptn.getParameters(), conf, inputJobInfo); + ptn.getParameters(), hiveConf, inputJobInfo); partInfo.setPartitionValues(InternalUtil.createPtnKeyValueMap(table, ptn)); partInfoList.add(partInfo); } @@ -135,7 +135,7 @@ private static InputJobInfo getInputJobInfo( //Non partitioned table HCatSchema schema = HCatUtil.extractSchema(table); PartInfo partInfo = extractPartInfo(schema, table.getTTable().getSd(), - table.getParameters(), conf, inputJobInfo); + table.getParameters(), hiveConf, inputJobInfo); partInfo.setPartitionValues(new HashMap()); partInfoList.add(partInfo); } @@ -149,13 +149,13 @@ private static InputJobInfo getInputJobInfo( } private static PartInfo extractPartInfo(HCatSchema schema, StorageDescriptor sd, - Map parameters, Configuration conf, + Map parameters, HiveConf hiveConf, InputJobInfo inputJobInfo) throws IOException { StorerInfo storerInfo = InternalUtil.extractStorerInfo(sd, parameters); Properties hcatProperties = new Properties(); - HiveStorageHandler storageHandler = HCatUtil.getStorageHandler(conf, storerInfo); + HiveStorageHandler storageHandler = HCatUtil.getStorageHandler(hiveConf, storerInfo); // copy the properties from storageHandler to jobProperties Map jobProperties = HCatUtil.getInputJobProperties(storageHandler, inputJobInfo);