[
https://issues.apache.org/jira/browse/HUDI-3204?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17890254#comment-17890254
]
Y Ethan Guo commented on HUDI-3204:
-----------------------------------
We may not want to fix this issue in the deprecated relation classes which is
only used by clustering (to be revisited and fixed). We should follow up to
remove the relation classes so we only maintain the new reader code path going
forward.
> Allow original partition column value to be retrieved when using
> TimestampBasedKeyGen
> -------------------------------------------------------------------------------------
>
> Key: HUDI-3204
> URL: https://issues.apache.org/jira/browse/HUDI-3204
> Project: Apache Hudi
> Issue Type: Bug
> Components: spark
> Affects Versions: 0.12.0
> Reporter: Yann Byron
> Assignee: Jonathan Vexler
> Priority: Critical
> Labels: hudi-on-call, pull-request-available, sev:critical
> Fix For: 1.1.0
>
> Original Estimate: 3h
> Time Spent: 1h
> Remaining Estimate: 1h
>
> {color:#172b4d}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.{color}
>
> {code:java}
> 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 {code}
>
>
--
This message was sent by Atlassian Jira
(v8.20.10#820010)