Details
-
Bug
-
Status: Open
-
Critical
-
Resolution: Unresolved
-
0.12.0
Description
Currently, b/c Spark by default omits partition values from the data files (instead encoding them into partition paths for partitioned tables), using `TimestampBasedKeyGenerator` w/ original timestamp based-column makes it impossible to retrieve the original value (reading from Spark) even though it's persisted in the data file as well.
import org.apache.hudi.DataSourceWriteOptions import org.apache.hudi.config.HoodieWriteConfig import org.apache.hudi.keygen.constant.KeyGeneratorOptions._ import org.apache.hudi.hive.MultiPartKeysValueExtractor val df = Seq((1, "z3", 30, "v1", "2018-09-23"), (2, "z3", 35, "v1", "2018-09-24")).toDF("id", "name", "age", "ts", "data_date") // mor df.write.format("hudi"). option(HoodieWriteConfig.TABLE_NAME, "issue_4417_mor"). option("hoodie.datasource.write.table.type", "MERGE_ON_READ"). option("hoodie.datasource.write.recordkey.field", "id"). option("hoodie.datasource.write.partitionpath.field", "data_date"). option("hoodie.datasource.write.precombine.field", "ts"). option("hoodie.datasource.write.keygenerator.class", "org.apache.hudi.keygen.TimestampBasedKeyGenerator"). option("hoodie.deltastreamer.keygen.timebased.timestamp.type", "DATE_STRING"). option("hoodie.deltastreamer.keygen.timebased.output.dateformat", "yyyy/MM/dd"). option("hoodie.deltastreamer.keygen.timebased.timezone", "GMT+8:00"). option("hoodie.deltastreamer.keygen.timebased.input.dateformat", "yyyy-MM-dd"). mode(org.apache.spark.sql.SaveMode.Append). save("file:///tmp/hudi/issue_4417_mor") +-------------------+--------------------+------------------+----------------------+--------------------+---+----+---+---+----------+ |_hoodie_commit_time|_hoodie_commit_seqno|_hoodie_record_key|_hoodie_partition_path| _hoodie_file_name| id|name|age| ts| data_date| +-------------------+--------------------+------------------+----------------------+--------------------+---+----+---+---+----------+ | 20220110172709324|20220110172709324...| 2| 2018/09/24|703e56d3-badb-40b...| 2| z3| 35| v1|2018-09-24| | 20220110172709324|20220110172709324...| 1| 2018/09/23|58fde2b3-db0e-464...| 1| z3| 30| v1|2018-09-23| +-------------------+--------------------+------------------+----------------------+--------------------+---+----+---+---+----------+ // can not query any data spark.read.format("hudi").load("file:///tmp/hudi/issue_4417_mor").where("data_date = '2018-09-24'") // still can not query any data spark.read.format("hudi").load("file:///tmp/hudi/issue_4417_mor").where("data_date = '2018/09/24'").show // cow df.write.format("hudi"). option(HoodieWriteConfig.TABLE_NAME, "issue_4417_cow"). option("hoodie.datasource.write.table.type", "COPY_ON_WRITE"). option("hoodie.datasource.write.recordkey.field", "id"). option("hoodie.datasource.write.partitionpath.field", "data_date"). option("hoodie.datasource.write.precombine.field", "ts"). option("hoodie.datasource.write.keygenerator.class", "org.apache.hudi.keygen.TimestampBasedKeyGenerator"). option("hoodie.deltastreamer.keygen.timebased.timestamp.type", "DATE_STRING"). option("hoodie.deltastreamer.keygen.timebased.output.dateformat", "yyyy/MM/dd"). option("hoodie.deltastreamer.keygen.timebased.timezone", "GMT+8:00"). option("hoodie.deltastreamer.keygen.timebased.input.dateformat", "yyyy-MM-dd"). mode(org.apache.spark.sql.SaveMode.Append). save("file:///tmp/hudi/issue_4417_cow") +-------------------+--------------------+------------------+----------------------+--------------------+---+----+---+---+----------+ |_hoodie_commit_time|_hoodie_commit_seqno|_hoodie_record_key|_hoodie_partition_path| _hoodie_file_name| id|name|age| ts| data_date| +-------------------+--------------------+------------------+----------------------+--------------------+---+----+---+---+----------+ | 20220110172721896|20220110172721896...| 2| 2018/09/24|81cc7819-a0d1-4e6...| 2| z3| 35| v1|2018/09/24| | 20220110172721896|20220110172721896...| 1| 2018/09/23|d428019b-a829-41a...| 1| z3| 30| v1|2018/09/23| +-------------------+--------------------+------------------+----------------------+--------------------+---+----+---+---+----------+ // can not query any data spark.read.format("hudi").load("file:///tmp/hudi/issue_4417_cow").where("data_date = '2018-09-24'").show // but 2018/09/24 works spark.read.format("hudi").load("file:///tmp/hudi/issue_4417_cow").where("data_date = '2018/09/24'").show
Attachments
Issue Links
- causes
-
HUDI-4161 Fix incorrect partition schema being passed to HadoopFsRelation
- Closed
- is a parent of
-
HUDI-8168 Allow original partition column value to be retrieved when using TimestampBasedKeyGen for 0.X datasource implementation
- Open
- is related to
-
HUDI-4430 Incorrect type casting while reading HUDI table created with CustomKeyGenerator and unixtimestamp paritioning field
- Open
-
HUDI-3902 Fallback to HadoopFsRelation for non-sophisticated COW use-cases
- Closed
-
HUDI-3338 Use custom relation instead of HadoopFsRelation
- Closed
- relates to
-
HUDI-6614 Partition pruning does not work for timestamp based keygenerator
- Closed
-
HUDI-8560 Revert HUDI-8036
- Patch Available
- links to