Details
-
Bug
-
Status: Closed
-
Major
-
Resolution: Fixed
-
0.7.0
-
hive 3.1.1
spark 2.4.5
hadoop 3.1.1
suse os
Description
step1: prepare raw DataFrame with DateType, and insert it to HudiMorTable
df_raw.withColumn("date", lit(Date.valueOf("2020-11-10")))
merge(df_raw, "bulk_insert", "huditest.bulkinsert_mor_10g")
step2: prepare update DataFrame with DateType, and upsert into HudiMorTable
df_update = sql("select * from huditest.bulkinsert_mor_10g_rt").withColumn("date", lit(Date.valueOf("2020-11-11")))
merge(df_update, "upsert", "huditest.bulkinsert_mor_10g")
step3: use hive-beeeline/ spark-sql query mor_rt table
use beeline/spark-sql execute statement select * from huditest.bulkinsert_mor_10g_rt where primary_key = 10000000;
then the follow error will occur:
java.lang.ClassCastExceoption: org.apache.hadoop.io.IntWritable cannot be cast to org.apache.hadoop.hive.serde2.io.DateWritableV2
Root cause analysis:
hudi use avro format to store log file, avro store DateType as INT(Type is INT but logcialType is date)。
when hudi read log file and convert avro INT type record to writable,logicalType is not respected which lead the dateType will cast to IntWritable。
seem: https://github.com/apache/hudi/blob/master/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieRealtimeRecordReaderUtils.java#L169
Modification plan: when cast avro INT type to writable, logicalType must be considerd
case INT:
if (schema.getLogicalType() != null && schema.getLogicalType().getName().equals("date"))
else
{ return new IntWritable((Integer) value); }
Attachments
Issue Links
- links to