diff options
author | Cheng Lian <lian@databricks.com> | 2016-06-23 23:11:46 -0700 |
---|---|---|
committer | Yin Huai <yhuai@databricks.com> | 2016-06-23 23:11:46 -0700 |
commit | 2d2f607bfae97f2681df24f48bb8b1b483c6b309 (patch) | |
tree | 917059d4730787edb05f1f5d6387ce5de0373d5e /sql/hive/src/main | |
parent | cc6778ee0bf4fa7a78abd30542c4a6f80ea371c5 (diff) | |
download | spark-2d2f607bfae97f2681df24f48bb8b1b483c6b309.tar.gz spark-2d2f607bfae97f2681df24f48bb8b1b483c6b309.tar.bz2 spark-2d2f607bfae97f2681df24f48bb8b1b483c6b309.zip |
[SPARK-13709][SQL] Initialize deserializer with both table and partition properties when reading partitioned tables
## What changes were proposed in this pull request?
When reading partitions of a partitioned Hive SerDe table, we only initializes the deserializer using partition properties. However, for SerDes like `AvroSerDe`, essential properties (e.g. Avro schema information) may be defined in table properties. We should merge both table properties and partition properties before initializing the deserializer.
Note that an individual partition may have different properties than the one defined in the table properties (e.g. partitions within a table can have different SerDes). Thus, for any property key defined in both partition and table properties, the value set in partition properties wins.
## How was this patch tested?
New test case added in `QueryPartitionSuite`.
Author: Cheng Lian <lian@databricks.com>
Closes #13865 from liancheng/spark-13709-partitioned-avro-table.
Diffstat (limited to 'sql/hive/src/main')
-rw-r--r-- | sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala | 17 |
1 files changed, 16 insertions, 1 deletions
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala index e49a235643..b4808fdbed 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala @@ -17,6 +17,10 @@ package org.apache.spark.sql.hive +import java.util.Properties + +import scala.collection.JavaConverters._ + import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{Path, PathFilter} import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants._ @@ -230,10 +234,21 @@ class HadoopTableReader( // Fill all partition keys to the given MutableRow object fillPartitionKeys(partValues, mutableRow) + val tableProperties = relation.tableDesc.getProperties + createHadoopRdd(tableDesc, inputPathStr, ifc).mapPartitions { iter => val hconf = broadcastedHiveConf.value.value val deserializer = localDeserializer.newInstance() - deserializer.initialize(hconf, partProps) + // SPARK-13709: For SerDes like AvroSerDe, some essential information (e.g. Avro schema + // information) may be defined in table properties. Here we should merge table properties + // and partition properties before initializing the deserializer. Note that partition + // properties take a higher priority here. For example, a partition may have a different + // SerDe as the one defined in table properties. + val props = new Properties(tableProperties) + partProps.asScala.foreach { + case (key, value) => props.setProperty(key, value) + } + deserializer.initialize(hconf, props) // get the table deserializer val tableSerDe = tableDesc.getDeserializerClass.newInstance() tableSerDe.initialize(hconf, tableDesc.getProperties) |