Open kirillklimenko opened 1 month ago
@kirillklimenko Normally PartialUpdateAvroPayload should avoid updating the columns which have null value.
@kirillklimenko Normally PartialUpdateAvroPayload should avoid updating the columns which have null value.
This is what I expect, however, it does not happen, which is strange.
@kirillklimenko I tried to mimic similar scenario but it is avoiding columns with null values. Can you come up with reproducible script.
@ad1happy2go, here is everything I have in my streamer: hudi-upsert-issue.zip
I'm running an application on AWS EMR 7.1 (Hudi 0.14.1) with AWS Glue sync:
{"sha512": "hash", "column1": "before", "column2": "before", "update_dt": "2024-01-01T00:00:00.000000"}
to the Kafka topic{"sha512": "hash", "column2": "after", "update_dt": "2024-02-02T00:00:00.000000"}
to the Kafka topicTested with:
use default;
select sha512, column1, column2 from default.bronze_ro; -- hash, before, before
select sha512, column1, column2 from default.bronze_rt; -- hash, null, after
spark.sql("select sha512, column1, column2 from default.bronze_ro").show() # hash, before, before
spark.sql("select sha512, column1, column2 from default.bronze_rt").show() # hash, null, after
@kirillklimenko We will look into it. Thanks for the details.
I was reading thread would this helphttps://soumilshah1995.blogspot.com/2023/05/hands-on-lab-unleashing-efficiency-and.html
+-----------+------------------+--------------+------------------+---------------------------+------+-------------------+
|customer_id|name |state |city |created_at |salary|_hoodie_commit_time|
+-----------+------------------+--------------+------------------+---------------------------+------+-------------------+
|1 |Mackenzie Martinez|Maryland |Lake Nicole |tdean@example.org |34853 |20230518073201741 |
|0 |Carolyn Calderon |South Carolina|West Geraldborough|porterchristina@example.org|69285 |20230518073201741 |
|4 |Jacob Barnes |Arizona |Michellehaven |lewisjennifer@example.net |48149 |20230518073201741 |
|3 |Elizabeth Jones |Michigan |Clarkborough |maurice87@example.net |null |20230518073201741 |
|2 |Sydney Ayala |Rhode Island |Port Patrickstad |kevin20@example.com |52640 |20230518073201741 |
+-----------+------------------+--------------+------------------+---------------------------+------+-------------------+
NEW RECORDS
+-----------+----+-----+--------+----------+-----+------+
|customer_id|name|state| city|created_at|email|salary|
+-----------+----+-----+--------+----------+-----+------+
| 1|null| CT|stamford| null| null| null|
+-----------+----+-----+--------+----------+-----+------+
AFTER UPSERT
+-----------+------------------+--------------+------------------+---------------------------+------+-------------------+
|customer_id|name |state |city |created_at |salary|_hoodie_commit_time|
+-----------+------------------+--------------+------------------+---------------------------+------+-------------------+
|1 |Mackenzie Martinez|CT |stamford |tdean@example.org |34853 |20230518073305092 |
|0 |Carolyn Calderon |South Carolina|West Geraldborough|porterchristina@example.org|69285 |20230518073201741 |
|4 |Jacob Barnes |Arizona |Michellehaven |lewisjennifer@example.net |48149 |20230518073201741 |
|3 |Elizabeth Jones |Michigan |Clarkborough |maurice87@example.net |null |20230518073201741 |
|2 |Sydney Ayala |Rhode Island |Port Patrickstad |kevin20@example.com |52640 |20230518073201741 |
+-----------+------------------+--------------+------------------+---------------------------+------+-------------------+
I have not tried streamer route I assume should be similar
I have not tried streamer route I assume should be similar
@soumilshah1995, I can confirm that org.apache.hudi.common.model.PartialUpdateAvroPayload
works as expected with Spark Structured Streaming, but doesn't work with Hudi Streamer.
I decided to stop trying to upsert with Hudi Streamer and rewrote everything to Spark.
But I still hope that the Hudi Streamer will be fixed, since this is the simplest and low-code way to stream to Hudi sources.
Describe the problem you faced
I'm using Hudi Streamer to ingest data from Kafka to Hudi MOR table.
If I produce the
{"sha512": "hash", "column1": "before", "column2": "before", "update_dt": "2024-01-01T00:00:00.000000"}
to the Kafka topic, I will get the following result in the Hudi table (after compaction):My goal is to update
column2
, without affectingcolumn1
by producing{"sha512": "hash", "column2": "after", "update_dt": "2024-02-02T00:00:00.000000"}
to the Kafka topic (after compaction).What I expect as a result (after compaction):
What I have as a result (after compaction):
Since I have
PartialUpdateAvroPayload
in my settings below, I expect a partial update, but I suspect it doesn't work because of"default": null
keyword in my schema definition.Could you suggest how can I update
column2
, without affectingcolumn1
with the current setup using Hudi Streamer?My
spark-submit
command:My
hudi.properties
file:My
transformer.sql
file:My
source.avsc
file:My
target.avsc
file:Environment Description
Hudi version : 0.14.1
Spark version : 3.4
Storage (HDFS/S3/GCS..) : S3
Running on Docker? (yes/no) : no