-
Notifications
You must be signed in to change notification settings - Fork 2.4k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
[SUPPORT] Hudi MOR high latency on data availability #11118
Comments
Thanks for raising this @sgcisco . I noticed you are using compact num.delta commits as 1. Any reason for the same. If we need to compact after every commit, then better we use COW table itself. |
@ad1happy2go thanks for your reply. We tried As another test attempt we tried to run a pipeline over several days but with lower ingestion rate 600Kb/s and the same Hudi and Spark configuration as above. The most time consuming stage is Over 3 days partitions latencies look as So in this case it is around 35-40Mb per minute, current Structured Streaming minibatch, and workers can go up to 35Gb and 32 cores. |
@sgcisco What is nature of your record key? Is it random id ? Building workload profile do the index lookup which is basically the join between the existing data with the incremental data to identify which records to be updated or inserted. |
@ad1happy2go record key looks as Increasing partitioning granularity from daily to hourly seems help to decrease latencies but not to solve the problem completely. In this case partitioning size goes down from 100Gb to 4.7Gb.
No, over 15h running job In this case, with low ingestion rate ~600Kb/s and hourly partitions, at Spark Structured streaming Which looks similar for latencies in written data |
Describe the problem you faced
Running a streaming solution with Kafka - Structured Streaming (PySpark) - Hudi (MOR tables) + AWS Glue+S3 we observed periodically growing latencies on data availability at Hudi.
Latencies were measured as difference between data generation
timestamp
and_hudi_commit_timestamp
and could go up to 30 min. Periodical manual checks for the latest available data pointstimestamps
, by running queries as described here https://hudi.apache.org/docs/0.13.1/querying_data#spark-snap-query, confirmed such delays.In case of using Spark with Hudi data read-out from Kafka had unstable rate
To exclude impact from any other components but Hudi we ran some experiments with the same configuration and ingestion settings but without Hudi and with a direct write on S3. It did not reveal any delays above 2 mins, where 1 min delay is always present due to Structured Streaming minibatch granularity. In this case a read-out Kafka rate was stable overtime.
Additional context
What tried
https://cwiki.apache.org/confluence/display/HUDI/FAQ#FAQ-HowdoItoavoidcreatingtonsofsmallfiles,
[SUPPORT] How to run Periodic Compaction? Multiple Tables - When no Upserts #2151 (comment)
We could get a target file size between 90-120Mb by downing
hoodie.copyonwrite.record.size.estimate
from 1024 to 100 and usingInline.compact=false and delta.commits=1 and async.compact=true and hoodie.merge.small.file.group.candidates.limit=20
but it did not have any impact on a latency.NUM_OR_TIME
as suggested here [SUPPORT] compaction.delta_seconds is not working for compaction strategy num_or_time #8975 (comment) with parameters below did not help to resolve a problemCurrent settings
As a trade-off we came up to the configuration below, which allows us to have relatively low latencies for 90th percentile and file size 40-90Mb
But still some records could go up to 30 min.
However the last config works relatively well for low ingestion rates up to 1.5Mb/s with a daily partitioning
partition_date=yyyy-MM-dd/
but stops work for the rates above 2.5 Mb/s even with more granular partitioningpartition_date=yyyy-MM-dd-HH/
Expected behavior
Since we use MOR tables:
Environment Description
Hudi version : 0.13.1
Spark version : 3.4.1
Hive version : 3.1
Hadoop version : EMR 6.13
Storage (HDFS/S3/GCS..) : S3
Running on Docker? (yes/no) : No
Hudi configuration
Spark configuration
The text was updated successfully, but these errors were encountered: