Closed ayush71994 closed 2 years ago
The issue seems to be happening only when the INSERT_DROP_DUPS_OPT_KEY flag is set to true. Looks like this config is being used for both:
As far as the behavior of the insert overwrite API is concerned it should always delete the partition and copy the incoming records. Drop duplicates should just pre-combine the input records.
@ayush71994 :
CC @satishkotha
@nsivabalan
thanks @am-cpp . @satishkotha : would appreciate if you can take a look at the issue.
While satish tries to investigate, one more question to narrow down the root cause. If you don't set https://hudi.apache.org/docs/configurations.html#INSERT_DROP_DUPS_OPT_KEY, is your records intact? I mean, new batch overwrites all data in matching partitions, but just that you will find duplicate records if any and your read does return only new records. Can you confirm this behavior?
@nsivabalan
folks, whats the next step here?
@am-cpp @ayush71994 . sorry, missed from the radar. Are you folks still interested in triaging this? I can assist you on this. Let me know.
I could not reproduce in latest master. https://gist.github.com/nsivabalan/23caa2f57c41bc9356ed7fa29590c147
Here is my understanding. INSERT_DROP_DUPES will delete records from incoming df with those matching in existing hudi table. when this is used along with INSERT_OVERRIDE operation, first insert_drop_dupes kicks in and so, possible some records from incoming batch will be dropped. and then INSERT_OVERRIDE is performed. and any matching partitions will be overritten. In my gist link, I did not use insert_drop_dupes for INSERT_OVERRIDE, just to show that it works. You need to set combine.before.insert/upsert to true to drop duplicates among incoming batch.
Here is the output if I use insert_drop_dupes with insert_override
+------+---------+---+ |typeId|recordKey|str| +------+---------+---+ |2 |key4 |mno| |1 |key1 |def| |3 |key5 |pqr| +------+---------+---+
As you could see, key2 is not present here, bcoz, it was dropped since it was already in hudi table.
@ayush71994 : Can you respond with your latest when you get a chance. would like to get to the bottom of this.
closing the issue we we could not reproduce. Feel free to re-open if you are still facing the issue. would be happy to assist
Tips before filing an issue
Have you gone through our FAQs?
Join the mailing list to engage in conversations and get faster support at dev-subscribe@hudi.apache.org.
If you have triaged this as a bug, then file an issue directly.
Describe the problem you faced
We are using EMR-5.33 with Hudi 0.7.0. Seeing an issue with the behaviour of insert_overwrite. From the documentation, we can use insert_overwrite to overwrite specific partitions. But what we are seeing is if the dataframe contains records that are present in the hudi table partition we are trying to overwrite, those records will be missing in the overwritten partition. In case all the records in the incoming dataframe match with the records in the table partition no write takes place. The partition is not overwritten. Incase of duplicate records or bad data it is not deleting the data already present in the partition. This behaviour seems different from what is described in the documentation
To Reproduce
Steps to reproduce the behavior:
Expected behavior
Expected insert_override to delete the older data and replace with the new dataframe without the duplicate records
Environment Description
Hudi version : 0.7.0
Spark version : 2.4.0
Hive version : version 1.2.2
Hadoop version : 2.10
EMR : 5.33
Storage (HDFS/S3/GCS..) : S3 and hive sync to Glue
Running on Docker? (yes/no) : Running on EMR using a fat jar
Additional context
Hudi config used
Replace Commits
From Hudi Cli
Contents of Replace commit
Stacktrace
No Errors