Details
-
Bug
-
Status: Resolved
-
Critical
-
Resolution: Unresolved
-
None
Description
Reading timestamp column from hudi and underlying parquet files in spark gives different results.
hudi properties:
hdfs dfs -cat /user/hive/warehouse/jira_expl.db/jiraissue_events/.hoodie/hoodie.properties #Properties saved on Tue May 11 17:17:22 EDT 2021 #Tue May 11 17:17:22 EDT 2021 hoodie.compaction.payload.class=org.apache.hudi.common.model.OverwriteWithLatestAvroPayload hoodie.table.name=jiraissue hoodie.archivelog.folder=archived hoodie.table.type=MERGE_ON_READ hoodie.table.version=1 hoodie.timeline.layout.version=1
Reading directly from parquet using Spark:
scala> val ji = spark.read.format("parquet").load("/user/hive/warehouse/jira_expl.db/jiraissue_events/*.parquet") ji: org.apache.spark.sql.DataFrame = [_hoodie_commit_time: string, _hoodie_commit_seqno: string ... 49 more fields]scala> ji.filter($"id" === 1237858).withColumn("inputfile", input_file_name()).select($"_hoodie_commit_time", $"_hoodie_commit_seqno", $"_hoodie_record_key", $"_hoodie_partition_path", $"_hoodie_file_name",$"resolutiondate", $"archiveddate", $"inputfile").show(false) +-------------------+----------------------+------------------+----------------------+-----------------------------------------------------------------------+--------------+------------+------------------------------------------------------------------------------------------------------------------------------------------------+ |_hoodie_commit_time|_hoodie_commit_seqno |_hoodie_record_key|_hoodie_partition_path|_hoodie_file_name |resolutiondate|archiveddate|inputfile | +-------------------+----------------------+------------------+----------------------+-----------------------------------------------------------------------+--------------+------------+------------------------------------------------------------------------------------------------------------------------------------------------+ |20210511171722 |20210511171722_7_13718|1237858.0 | |832cf07f-637b-4a4c-ab08-6929554f003a-0_7-98-5106_20210511171722.parquet|null |null |hdfs://nameservice1/user/hive/warehouse/jira_expl.db/jiraissue_events/832cf07f-637b-4a4c-ab08-6929554f003a-0_7-98-5106_20210511171722.parquet | |20210511171722 |20210511171722_7_13718|1237858.0 | |832cf07f-637b-4a4c-ab08-6929554f003a-0_7-98-5106_20210511171722.parquet|null |null |hdfs://nameservice1/user/hive/warehouse/jira_expl.db/jiraissue_events/832cf07f-637b-4a4c-ab08-6929554f003a-0_8-1610-78711_20210511173615.parquet%7C +-------------------+----------------------+------------------+----------------------+-----------------------------------------------------------------------+--------------+------------+------------------------------------------------------------------------------------------------------------------------------------------------+
Reading `hudi` using Spark:
scala> val jih = spark.read.format("org.apache.hudi").load("/user/hive/warehouse/jira_expl.db/jiraissue_events") jih: org.apache.spark.sql.DataFrame = [_hoodie_commit_time: string, _hoodie_commit_seqno: string ... 49 more fields]scala> jih.filter($"id" === 1237858).select($"_hoodie_commit_time", $"_hoodie_commit_seqno", $"_hoodie_record_key", $"_hoodie_partition_path", $"_hoodie_file_name",$"resolutiondate", $"archiveddate").show(false) +-------------------+----------------------+------------------+----------------------+-----------------------------------------------------------------------+-------------------+-------------------+ |_hoodie_commit_time|_hoodie_commit_seqno |_hoodie_record_key|_hoodie_partition_path|_hoodie_file_name |resolutiondate |archiveddate | +-------------------+----------------------+------------------+----------------------+-----------------------------------------------------------------------+-------------------+-------------------+ |20210511171722 |20210511171722_7_13718|1237858.0 | |832cf07f-637b-4a4c-ab08-6929554f003a-0_7-98-5106_20210511171722.parquet|2018-07-30 14:58:52|1969-12-31 19:00:00| +-------------------+----------------------+------------------+----------------------+-----------------------------------------------------------------------+-------------------+-------------------+