1
0

[HUDI-1225] Fix: Avro Date logical type not handled correctly when converting to Spark Row (#2047)

This commit is contained in:
Thinking Chen
2020-08-29 16:16:42 +08:00
committed by GitHub
parent 0360bef217
commit 6b417d1a86
2 changed files with 63 additions and 4 deletions

View File

@@ -30,6 +30,7 @@ import org.apache.avro.generic.{GenericData, GenericFixed, GenericRecord}
import org.apache.spark.sql.Row
import org.apache.spark.sql.avro.{IncompatibleSchemaException, SchemaConverters}
import org.apache.spark.sql.catalyst.expressions.GenericRow
import org.apache.spark.sql.catalyst.util.DateTimeUtils
import org.apache.spark.sql.types._
import scala.collection.JavaConverters._
@@ -110,10 +111,9 @@ object AvroConversionHelper {
if (item == null) {
null
} else {
if (item.isInstanceOf[Integer]) {
new Date(item.asInstanceOf[Integer].longValue())
} else {
new Date(item.asInstanceOf[Long])
item match {
case integer: Integer => DateTimeUtils.toJavaDate(integer)
case _ => new Date(item.asInstanceOf[Long])
}
}
case (TimestampType, LONG) =>