[
https://issues.apache.org/jira/browse/HUDI-3204?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17471979#comment-17471979
]
Yann Byron commented on HUDI-3204:
----------------------------------
yeah, the difference between the two table types is wried. I'll modify the MOR
type's behavior to keep it same with COW table. But when finish this, it will
not be compatible with the existing MOR table.
> spark on TimestampBasedKeyGenerator has no result when query by partition
> column
> --------------------------------------------------------------------------------
>
> Key: HUDI-3204
> URL: https://issues.apache.org/jira/browse/HUDI-3204
> Project: Apache Hudi
> Issue Type: Bug
> Components: Spark Integration
> Reporter: Yann Byron
> Assignee: Yann Byron
> Priority: Major
> Fix For: 0.11.0
>
>
>
> {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.1#820001)