apache / hudi

Upserts, Deletes And Incremental Processing on Big Data.
https://hudi.apache.org/
Apache License 2.0
5.23k stars 2.39k forks source link

[SUPPORT] failed to read timestamp from hive #7724

Open szingerpeter opened 1 year ago

szingerpeter commented 1 year ago

Hy,

I'm using Hudi CLI version 1.0; hudi version 0.11.0; Spark version 3.2.1-amzn-0 and Hive version 3.1.3-amzn-0.

the error i'm getting:

 java.lang.ClassCastException: org.apache.hadoop.io.LongWritable cannot be cast to org.apache.hadoop.hive.serde2.io.TimestampWritable
        at org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableTimestampObjectInspector.getPrimitiveJavaObject(WritableTimestampObjectInspector.java:39)

write settings:

hoodie.datasource.hive_sync.support_timestamp: True

spark settings:

spark.serializer: org.apache.spark.serializer.KryoSerializer
spark.sql.hive.convertMetastoreParquet: false
hive.input.format: org.apache.hadoop.hive.ql.io.HiveInputFormat
spark.sql.avro.datetimeRebaseModeInWrite: CORRECTED

i've seen multiple other issues reporting the same error and PRs, but couldn't figure out what would be the right solution #2509 #2544 #2869 #3391 #3378 #6232

7730

also, this link on the FAQ page doesn't work

szingerpeter commented 1 year ago

i tried setting

spark.sql.avro.datetimeRebaseModeInWrite=LEGACY

as well, but i get the same error:

Caused by: java.lang.ClassCastException: org.apache.hadoop.io.LongWritable cannot be cast to org.apache.hadoop.hive.serde2.io.TimestampWritable
        at org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableTimestampObjectInspector.getPrimitiveJavaObject(WritableTimestampObjectInspector.java:39)

the reason i need to set spark.sql.avro.datetimeRebaseModeInWrite is, because i was getting the following error:

org.apache.spark.SparkUpgradeException: You may get a different result due to the upgrading of Spark 3.0: 
writing dates before 1582-10-15 or timestamps before 1900-01-01T00:00:00Z into Avro
files can be dangerous, as the files may be read by Spark 2.x or legacy versions of Hive
later, which uses a legacy hybrid calendar that is different from Spark 3.0+'s Proleptic
Gregorian calendar. See more details in SPARK-31404. You can set spark.sql.avro.datetimeRebaseModeInWrite to 'LEGACY' to
rebase the datetime values w.r.t. the calendar difference during writing, to get maximum
interoperability. Or set spark.sql.avro.datetimeRebaseModeInWrite to 'CORRECTED' to write the datetime values as it is,
if you are 100% sure that the written files will only be read by Spark 3.0+ or other
systems that use Proleptic Gregorian calendar.
danny0405 commented 1 year ago

What is your timestamp precision, only micro-seconds with precision 6 is synced with Timestamp type in Hive.

szingerpeter commented 1 year ago

it's got second-precision

szingerpeter commented 1 year ago

i get the same error even if i don't set spark.sql.avro.datetimeRebaseModeInWrite.

Note:

spark.read.format('hudi').load(PATH) works fine spark.sql('select * from TABLE') fails (and also queries in hive)

szingerpeter commented 1 year ago

@danny0405 , I tried with artificially creating micro-second precision timestamps (added the suffix .000001 to each value and casted it to timestamp). saving them results in exactly the same error

szingerpeter commented 1 year ago

update:

if i don't specify the table in advance in hive, but let hudi sync and create a new table it works with artificially adding the .000001.

I know it's becoming a bit of a spark-specific question, but do you have an idea on how to cast to timestamp with MS precision with 0 MS?

spark.createDataFrame([['2022-01-01 01:01:01.000000']]).withColumn('test', F.col('_1').cast('timestamp')).show(10, False)

+--------------------------+-------------------+
|_1                        |test               |
+--------------------------+-------------------+
|2022-01-01 01:01:01.000000|2022-01-01 01:01:01|
+--------------------------+-------------------+

spark.createDataFrame([['2022-01-01 01:01:01.000000']]).withColumn('test', F.to_timestamp(F.col('_1'), 'yyyy-MM-dd HH:mm:ss.SSSSSS')).show(10, False)
+--------------------------+-------------------+
|_1                        |test               |
+--------------------------+-------------------+
|2022-01-01 01:01:01.000000|2022-01-01 01:01:01|
+--------------------------+-------------------+
danny0405 commented 1 year ago

not sure but explicit cast expr like cast(f0 as timestamp(0)) work here?

szingerpeter commented 1 year ago

it's not supported.

pyspark.sql.utils.ParseException:
DataType timestamp(0) is not supported.(line 1, pos 11)

== SQL ==
CAST(_1 AS TIMESTAMP(0))
-----------^^^
spark.createDataFrame([['2022-01-01 01:01:01.000000']]).withColumn('test', F.expr('CAST(_1 AS TIMESTAMP(0))'))

I guess for hudi to save the column as of type timestamp, the dataframe must have a timestamp type as well rather than a timestamp formatted string, right?

szingerpeter commented 1 year ago

in the end, setting spark.sql.parquet.outputTimestampType: TIMESTAMP_MICROS solved the problem with not defining the table in hive beforehand.

i see when hudi creates the table in hive, it has some additional properties like spark.sql.sources.schema.part.0 - i wonder how it's being used?

danny0405 commented 1 year ago

It's used for the spark sql schema inference, glad to hear that you solved the problem, so we can close this issue now?

szingerpeter commented 1 year ago

well, the issue still persists if i try to query the table from hive (beeline).

Error: java.io.IOException: org.apache.hadoop.hive.ql.metadata.HiveException: java.lang.ClassCastException: org.apache.hadoop.io.LongWritable cannot be cast to org.apache.hadoop.hive.serde2.io.TimestampWritableV2 (state=,code=0)

any suggestion to solve this part of the problem?

xicm commented 1 year ago

3391 can solve the problem, I tested, and I solved the conflicts on master branch in #7173, you can port this pr and have a test.