Skip to content

Commit e6d1406

Browse files
jeanlynmarmbrus
authored andcommitted
[SPARK-5498][SQL]fix query exception when partition schema does not match table schema
In hive,the schema of partition may be difference from the table schema.When we use spark-sql to query the data of partition which schema is difference from the table schema,we will get the exceptions as the description of the [jira](https://issues.apache.org/jira/browse/SPARK-5498) .For example: * We take a look of the schema for the partition and the table ```sql DESCRIBE partition_test PARTITION (dt='1'); id int None name string None dt string None # Partition Information # col_name data_type comment dt string None ``` ``` DESCRIBE partition_test; OK id bigint None name string None dt string None # Partition Information # col_name data_type comment dt string None ``` * run the sql ```sql SELECT * FROM partition_test where dt='1'; ``` we will get the cast exception `java.lang.ClassCastException: org.apache.spark.sql.catalyst.expressions.MutableLong cannot be cast to org.apache.spark.sql.catalyst.expressions.MutableInt` Author: jeanlyn <jeanlyn92@gmail.com> Closes #4289 from jeanlyn/schema and squashes the following commits: 9c8da74 [jeanlyn] fix style b41d6b9 [jeanlyn] fix compile errors 07d84b6 [jeanlyn] Merge branch 'master' into schema 535b0b6 [jeanlyn] reduce conflicts d6c93c5 [jeanlyn] fix bug 1e8b30c [jeanlyn] fix code style 0549759 [jeanlyn] fix code style c879aa1 [jeanlyn] clean the code 2a91a87 [jeanlyn] add more test case and clean the code 12d800d [jeanlyn] fix code style 63d170a [jeanlyn] fix compile problem 7470901 [jeanlyn] reduce conflicts afc7da5 [jeanlyn] make getConvertedOI compatible between 0.12.0 and 0.13.1 b1527d5 [jeanlyn] fix type mismatch 10744ca [jeanlyn] Insert a space after the start of the comment 3b27af3 [jeanlyn] SPARK-5498:fix bug when query the data when partition schema does not match table schema
1 parent 8c3b005 commit e6d1406

File tree

4 files changed

+84
-14
lines changed

4 files changed

+84
-14
lines changed

sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala

Lines changed: 27 additions & 10 deletions
Original file line numberDiff line numberDiff line change
@@ -25,7 +25,7 @@ import org.apache.hadoop.hive.ql.exec.Utilities
2525
import org.apache.hadoop.hive.ql.metadata.{Partition => HivePartition, Table => HiveTable}
2626
import org.apache.hadoop.hive.ql.plan.{PlanUtils, TableDesc}
2727
import org.apache.hadoop.hive.serde2.Deserializer
28-
import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector
28+
import org.apache.hadoop.hive.serde2.objectinspector.{ObjectInspectorConverters, StructObjectInspector}
2929
import org.apache.hadoop.hive.serde2.objectinspector.primitive._
3030
import org.apache.hadoop.io.Writable
3131
import org.apache.hadoop.mapred.{FileInputFormat, InputFormat, JobConf}
@@ -116,7 +116,7 @@ class HadoopTableReader(
116116
val hconf = broadcastedHiveConf.value.value
117117
val deserializer = deserializerClass.newInstance()
118118
deserializer.initialize(hconf, tableDesc.getProperties)
119-
HadoopTableReader.fillObject(iter, deserializer, attrsWithIndex, mutableRow)
119+
HadoopTableReader.fillObject(iter, deserializer, attrsWithIndex, mutableRow, deserializer)
120120
}
121121

122122
deserializedHadoopRDD
@@ -189,9 +189,13 @@ class HadoopTableReader(
189189
val hconf = broadcastedHiveConf.value.value
190190
val deserializer = localDeserializer.newInstance()
191191
deserializer.initialize(hconf, partProps)
192+
// get the table deserializer
193+
val tableSerDe = tableDesc.getDeserializerClass.newInstance()
194+
tableSerDe.initialize(hconf, tableDesc.getProperties)
192195

193196
// fill the non partition key attributes
194-
HadoopTableReader.fillObject(iter, deserializer, nonPartitionKeyAttrs, mutableRow)
197+
HadoopTableReader.fillObject(iter, deserializer, nonPartitionKeyAttrs,
198+
mutableRow, tableSerDe)
195199
}
196200
}.toSeq
197201

@@ -261,25 +265,36 @@ private[hive] object HadoopTableReader extends HiveInspectors {
261265
* Transform all given raw `Writable`s into `Row`s.
262266
*
263267
* @param iterator Iterator of all `Writable`s to be transformed
264-
* @param deserializer The `Deserializer` associated with the input `Writable`
268+
* @param rawDeser The `Deserializer` associated with the input `Writable`
265269
* @param nonPartitionKeyAttrs Attributes that should be filled together with their corresponding
266270
* positions in the output schema
267271
* @param mutableRow A reusable `MutableRow` that should be filled
272+
* @param tableDeser Table Deserializer
268273
* @return An `Iterator[Row]` transformed from `iterator`
269274
*/
270275
def fillObject(
271276
iterator: Iterator[Writable],
272-
deserializer: Deserializer,
277+
rawDeser: Deserializer,
273278
nonPartitionKeyAttrs: Seq[(Attribute, Int)],
274-
mutableRow: MutableRow): Iterator[Row] = {
279+
mutableRow: MutableRow,
280+
tableDeser: Deserializer): Iterator[Row] = {
281+
282+
val soi = if (rawDeser.getObjectInspector.equals(tableDeser.getObjectInspector)) {
283+
rawDeser.getObjectInspector.asInstanceOf[StructObjectInspector]
284+
} else {
285+
HiveShim.getConvertedOI(
286+
rawDeser.getObjectInspector,
287+
tableDeser.getObjectInspector).asInstanceOf[StructObjectInspector]
288+
}
275289

276-
val soi = deserializer.getObjectInspector().asInstanceOf[StructObjectInspector]
277290
val (fieldRefs, fieldOrdinals) = nonPartitionKeyAttrs.map { case (attr, ordinal) =>
278291
soi.getStructFieldRef(attr.name) -> ordinal
279292
}.unzip
280293

281-
// Builds specific unwrappers ahead of time according to object inspector types to avoid pattern
282-
// matching and branching costs per row.
294+
/**
295+
* Builds specific unwrappers ahead of time according to object inspector
296+
* types to avoid pattern matching and branching costs per row.
297+
*/
283298
val unwrappers: Seq[(Any, MutableRow, Int) => Unit] = fieldRefs.map {
284299
_.getFieldObjectInspector match {
285300
case oi: BooleanObjectInspector =>
@@ -316,9 +331,11 @@ private[hive] object HadoopTableReader extends HiveInspectors {
316331
}
317332
}
318333

334+
val converter = ObjectInspectorConverters.getConverter(rawDeser.getObjectInspector, soi)
335+
319336
// Map each tuple to a row object
320337
iterator.map { value =>
321-
val raw = deserializer.deserialize(value)
338+
val raw = converter.convert(rawDeser.deserialize(value))
322339
var i = 0
323340
while (i < fieldRefs.length) {
324341
val fieldValue = soi.getStructFieldData(raw, fieldRefs(i))

sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertIntoHiveTableSuite.scala

Lines changed: 42 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -32,9 +32,12 @@ import org.apache.spark.sql.hive.test.TestHive._
3232

3333
case class TestData(key: Int, value: String)
3434

35+
case class ThreeCloumntable(key: Int, value: String, key1: String)
36+
3537
class InsertIntoHiveTableSuite extends QueryTest with BeforeAndAfter {
3638
import org.apache.spark.sql.hive.test.TestHive.implicits._
3739

40+
3841
val testData = TestHive.sparkContext.parallelize(
3942
(1 to 100).map(i => TestData(i, i.toString))).toDF()
4043

@@ -186,4 +189,43 @@ class InsertIntoHiveTableSuite extends QueryTest with BeforeAndAfter {
186189

187190
sql("DROP TABLE hiveTableWithStructValue")
188191
}
192+
193+
test("SPARK-5498:partition schema does not match table schema") {
194+
val testData = TestHive.sparkContext.parallelize(
195+
(1 to 10).map(i => TestData(i, i.toString))).toDF()
196+
testData.registerTempTable("testData")
197+
198+
val testDatawithNull = TestHive.sparkContext.parallelize(
199+
(1 to 10).map(i => ThreeCloumntable(i, i.toString,null))).toDF()
200+
201+
val tmpDir = Files.createTempDir()
202+
sql(s"CREATE TABLE table_with_partition(key int,value string) PARTITIONED by (ds string) location '${tmpDir.toURI.toString}' ")
203+
sql("INSERT OVERWRITE TABLE table_with_partition partition (ds='1') SELECT key,value FROM testData")
204+
205+
// test schema the same between partition and table
206+
sql("ALTER TABLE table_with_partition CHANGE COLUMN key key BIGINT")
207+
checkAnswer(sql("select key,value from table_with_partition where ds='1' "),
208+
testData.collect.toSeq
209+
)
210+
211+
// test difference type of field
212+
sql("ALTER TABLE table_with_partition CHANGE COLUMN key key BIGINT")
213+
checkAnswer(sql("select key,value from table_with_partition where ds='1' "),
214+
testData.collect.toSeq
215+
)
216+
217+
// add column to table
218+
sql("ALTER TABLE table_with_partition ADD COLUMNS(key1 string)")
219+
checkAnswer(sql("select key,value,key1 from table_with_partition where ds='1' "),
220+
testDatawithNull.collect.toSeq
221+
)
222+
223+
// change column name to table
224+
sql("ALTER TABLE table_with_partition CHANGE COLUMN key keynew BIGINT")
225+
checkAnswer(sql("select keynew,value from table_with_partition where ds='1' "),
226+
testData.collect.toSeq
227+
)
228+
229+
sql("DROP TABLE table_with_partition")
230+
}
189231
}

sql/hive/v0.12.0/src/main/scala/org/apache/spark/sql/hive/Shim12.scala

Lines changed: 8 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -34,7 +34,7 @@ import org.apache.hadoop.hive.ql.plan.{CreateTableDesc, FileSinkDesc, TableDesc}
3434
import org.apache.hadoop.hive.ql.processors._
3535
import org.apache.hadoop.hive.ql.stats.StatsSetupConst
3636
import org.apache.hadoop.hive.serde2.{ColumnProjectionUtils, Deserializer, io => hiveIo}
37-
import org.apache.hadoop.hive.serde2.objectinspector.{ObjectInspector, PrimitiveObjectInspector}
37+
import org.apache.hadoop.hive.serde2.objectinspector.{ObjectInspectorConverters, ObjectInspector, PrimitiveObjectInspector}
3838
import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector.PrimitiveCategory
3939
import org.apache.hadoop.hive.serde2.objectinspector.primitive.{HiveDecimalObjectInspector, PrimitiveObjectInspectorFactory}
4040
import org.apache.hadoop.hive.serde2.typeinfo.{TypeInfo, TypeInfoFactory}
@@ -210,7 +210,7 @@ private[hive] object HiveShim {
210210

211211
def getDataLocationPath(p: Partition) = p.getPartitionPath
212212

213-
def getAllPartitionsOf(client: Hive, tbl: Table) = client.getAllPartitionsForPruner(tbl)
213+
def getAllPartitionsOf(client: Hive, tbl: Table) = client.getAllPartitionsForPruner(tbl)
214214

215215
def compatibilityBlackList = Seq(
216216
"decimal_.*",
@@ -244,6 +244,12 @@ private[hive] object HiveShim {
244244
}
245245
}
246246

247+
def getConvertedOI(
248+
inputOI: ObjectInspector,
249+
outputOI: ObjectInspector): ObjectInspector = {
250+
ObjectInspectorConverters.getConvertedOI(inputOI, outputOI, true)
251+
}
252+
247253
def prepareWritable(w: Writable): Writable = {
248254
w
249255
}

sql/hive/v0.13.1/src/main/scala/org/apache/spark/sql/hive/Shim13.scala

Lines changed: 7 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -17,6 +17,7 @@
1717

1818
package org.apache.spark.sql.hive
1919

20+
import java.util
2021
import java.util.{ArrayList => JArrayList}
2122
import java.util.Properties
2223
import java.rmi.server.UID
@@ -38,7 +39,7 @@ import org.apache.hadoop.hive.ql.processors.CommandProcessorFactory
3839
import org.apache.hadoop.hive.serde.serdeConstants
3940
import org.apache.hadoop.hive.serde2.typeinfo.{TypeInfo, DecimalTypeInfo, TypeInfoFactory}
4041
import org.apache.hadoop.hive.serde2.objectinspector.primitive.{HiveDecimalObjectInspector, PrimitiveObjectInspectorFactory}
41-
import org.apache.hadoop.hive.serde2.objectinspector.{PrimitiveObjectInspector, ObjectInspector}
42+
import org.apache.hadoop.hive.serde2.objectinspector.{ObjectInspectorConverters, PrimitiveObjectInspector, ObjectInspector}
4243
import org.apache.hadoop.hive.serde2.{Deserializer, ColumnProjectionUtils}
4344
import org.apache.hadoop.hive.serde2.{io => hiveIo}
4445
import org.apache.hadoop.hive.serde2.avro.AvroGenericRecordWritable
@@ -400,7 +401,11 @@ private[hive] object HiveShim {
400401
Decimal(hdoi.getPrimitiveJavaObject(data).bigDecimalValue(), hdoi.precision(), hdoi.scale())
401402
}
402403
}
403-
404+
405+
def getConvertedOI(inputOI: ObjectInspector, outputOI: ObjectInspector): ObjectInspector = {
406+
ObjectInspectorConverters.getConvertedOI(inputOI, outputOI)
407+
}
408+
404409
/*
405410
* Bug introduced in hive-0.13. AvroGenericRecordWritable has a member recordReaderID that
406411
* is needed to initialize before serialization.

0 commit comments

Comments
 (0)