mgstahl-sophos opened a new issue, #13995:
URL: https://github.com/apache/hudi/issues/13995
### Describe the problem you faced
We’ve observed that hudi-1.0.2 (tv=6) is creating significantly more file
groups that hudi-0.14 on the same input data when both are running similar
configuration, and would like some help understanding why.
To compare streamer performance between hudi-0.14 and hudi-1.0 we ran two
streamer applications in `--continuous` mode reading from the same kafka topic,
each reading 50 million events per batch starting at the same offsets (one hour
ago so that each would read the full 50M events per batch).
The configuration is the same except for two hudi-1x only properties:
`_hoodie.record.size.estimator.max.commits: 100`
`hoodie.write.table.version: 6`
Since hudi-0.14 uses all commits in the active timeline when calculating the
average record size but by default hudi-1.x only uses the most recent 5
commits, we set `_hoodie.record.size.estimator.max.commits=100` hoping to get
equivalent behavior in the average record size computations used to determine
whether to grow existing small files or create new files.
We suspect something in this logic is causing the differences, but cannot
explain how.
Doing a commit-by-commit comparison between the two apps we do see that
there are significant differences in the AvgRecordSize computations:
```
# Commit-by-commit AvgRecordSize for hudi-0.14 (LHS) v hudi-1 (tv=6) (RHS)
diff -y <(cat
/tmp/hudi-ingest-testperfhudi014registryg-driver-logs-at-start.txt| grep
AvgRecordSize | awk '{print $NF}') \
<(cat
/tmp/hudi-ingest-testperfhudi1tv6registryg-driver-logs-at-start.txt | grep
AvgRecordSize | awk '{print $NF}') | head -21
1024 1024
163 | 1024
131 | 1024
127 | 1024
124 | 1024
122 | 1024
120 | 1024
119 | 1024
118 | 1024
118 | 1024
118 | 1024
117 | 1024
116 | 1024
117 | 1024
118 | 1024
119 | 267
118 | 267
118 | 267
118 | 267
117 | 267
116 | 267
```
After the test we downloaded and analyzed the timeline metadata with jq. We
calculated statistics for the 1st commit of each app, then the 2nd commit of
each app, and so on, until all commits for both apps were processed. We
compared file and record statistics between the two apps for each commit, and
cumulative statististics for the sequence of commits. The averageRecordSize is
calculated in jq as the ceiling of totalFileBytes / totalRecords.
Comparing the 1st commits for each app showed mostly the same stats, only
differing in total file size (~1 MB) .
```
% jq -n -L ./scripts/timeline/ --slurpfile hudi014
/tmp/timeline-stats-testperfhudi014registryg.json \
--slurpfile hudi1tv6 /tmp/timeline-stats-testperfhudi1tv6registryg.json '
import "timeline-functions" as timeline;
timeline::compare_stat_files($hudi014[0]; $hudi1tv6[0]; "hudi014";
"hudi1tv6")' \
| jq '.|.cumulativeByCommit[0]'
{
"commitNumber": 1,
"hudi014": {
"commit": "20250925151905376",
"totalFileGroupCount": 949,
"totalFileCount": 949,
"totalFileBytes": 8109414647,
"totalInsertCount": 50000000,
"totalRecordCount": 50000000,
"totalUpsertTime": 0,
"averageFileSizeBytes": 8545221,
"averageInsertCount": 52688,
"averageRecordCount": 52688,
"averageRecordSize": 163,
"averageUpsertTime": 0
},
"hudi1tv6": {
"commit": "20250925161616841",
"totalFileGroupCount": 949,
"totalFileCount": 949,
"totalFileBytes": 8108059527,
"totalInsertCount": 50000000,
"totalRecordCount": 50000000,
"totalUpsertTime": 0,
"averageFileSizeBytes": 8543793,
"averageInsertCount": 52688,
"averageRecordCount": 52688,
"averageRecordSize": 163,
"averageUpsertTime": 0
},
"differences": {
"totalFileGroupCount": 0,
"totalFileCount": 0,
"totalFileBytes": -1355120,
"totalInsertCount": 0,
"totalRecordCount": 0,
"totalUpsertTime": 0,
"averageFileSizeBytes": -1428,
"averageInsertCount": 0,
"averageRecordCount": 0,
"averageRecordSize": 0,
"averageUpsertTime": 0
},
"ratios": {
"totalFileGroupCount": 1,
"totalFileCount": 1,
"totalFileBytes": 1,
"totalInsertCount": 1,
"totalRecordCount": 1,
"totalUpsertTime": null,
"averageFileSizeBytes": 1,
"averageInsertCount": 1,
"averageRecordCount": 1,
"averageRecordSize": 1,
"averageUpsertTime": null
}
}
```
Subsequent commits show hudi-1 producing significantly more file groups than
hudi-0.14 (In the 2nd commit 29% more and 18% more files).
Here is the 2nd commit comparison:
```
% jq -n -L ./scripts/timeline/ --slurpfile hudi014
/tmp/timeline-stats-testperfhudi014registryg.json \
--slurpfile hudi1tv6 /tmp/timeline-stats-testperfhudi1tv6registryg.json '
import "timeline-functions" as timeline;
timeline::compare_stat_files($hudi014[0]; $hudi1tv6[0]; "hudi014";
"hudi1tv6")' \
| jq '.|.cumulativeByCommit[1]'
{
"commitNumber": 2,
"hudi014": {
"commit": "20250925152015676",
"totalFileGroupCount": 1007,
"totalFileCount": 1606,
"totalFileBytes": 16105469533,
"totalInsertCount": 100000000,
"totalRecordCount": 111410769,
"totalUpsertTime": 3327834,
"averageFileSizeBytes": 10028313,
"averageInsertCount": 62267,
"averageRecordCount": 69372,
"averageRecordSize": 145,
"averageUpsertTime": 2073
},
"hudi1tv6": {
"commit": "20250925161737368",
"totalFileGroupCount": 1298,
"totalFileCount": 1897,
"totalFileBytes": 17648874528,
"totalInsertCount": 100000000,
"totalRecordCount": 111410769,
"totalUpsertTime": 2678311,
"averageFileSizeBytes": 9303572,
"averageInsertCount": 52715,
"averageRecordCount": 58730,
"averageRecordSize": 159,
"averageUpsertTime": 1412
},
"differences": {
"totalFileGroupCount": 291,
"totalFileCount": 291,
"totalFileBytes": 1543404995,
"totalInsertCount": 0,
"totalRecordCount": 0,
"totalUpsertTime": -649523,
"averageFileSizeBytes": -724741,
"averageInsertCount": -9552,
"averageRecordCount": -10642,
"averageRecordSize": 14,
"averageUpsertTime": -661
},
"ratios": {
"totalFileGroupCount": 1.29,
"totalFileCount": 1.18,
"totalFileBytes": 1.1,
"totalInsertCount": 1,
"totalRecordCount": 1,
"totalUpsertTime": 0.8,
"averageFileSizeBytes": 0.93,
"averageInsertCount": 0.85,
"averageRecordCount": 0.85,
"averageRecordSize": 1.1,
"averageUpsertTime": 0.68
}
}
```
After all commits, we’ve accumulated almost 3x the number of fileGroups in
hudi-1x than hudi-0.14 and 21% more files.
```
% jq -n -L ./scripts/timeline/ --slurpfile hudi014
/tmp/timeline-stats-testperfhudi014registryg.json \
--slurpfile hudi1tv6 /tmp/timeline-stats-testperfhudi1tv6registryg.json '
import "timeline-functions" as timeline;
timeline::compare_stat_files($hudi014[0]; $hudi1tv6[0]; "hudi014";
"hudi1tv6")' \
| jq '.|del(.cumulativeByCommit)'
{
"summary": {
"hudi014": {
"commitCount": 31,
"elapsed": "PT50M34S",
"totalFileGroupCount": 2025,
"totalFileCount": 20059,
"totalFileBytes": 508282998165,
"totalInsertCount": 1236358798,
"totalRecordCount": 4322931214,
"totalUpsertTime": 222288008,
"averageFileSizeBytes": 25339399,
"averageInsertCount": 61637,
"averageRecordCount": 215511,
"averageRecordSize": 118,
"averageUpsertTime": 11082
},
"hudi1tv6": {
"commitCount": 31,
"elapsed": "PT45M59S",
"totalFileGroupCount": 6077,
"totalFileCount": 24305,
"totalFileBytes": 448165211397,
"totalInsertCount": 1236358798,
"totalRecordCount": 3496849523,
"totalUpsertTime": 180974805,
"averageFileSizeBytes": 18439219,
"averageInsertCount": 50869,
"averageRecordCount": 143874,
"averageRecordSize": 129,
"averageUpsertTime": 7446
}
},
"differences": {
"totalFileGroupCount": 4052,
"totalFileCount": 4246,
"totalFileBytes": -60117786768,
"totalInsertCount": 0,
"totalRecordCount": -826081691,
"totalUpsertTime": -41313203,
"averageFileSizeBytes": -6900180,
"averageInsertCount": -10768,
"averageRecordCount": -71637,
"averageRecordSize": 11,
"averageUpsertTime": -3636
},
"ratios": {
"totalFileGroupCount": 3,
"totalFileCount": 1.21,
"totalFileBytes": 0.88,
"totalInsertCount": 1,
"totalRecordCount": 0.81,
"totalUpsertTime": 0.81,
"averageFileSizeBytes": 0.73,
"averageInsertCount": 0.83,
"averageRecordCount": 0.67,
"averageRecordSize": 1.09,
"averageUpsertTime": 0.67
}
}
```
### To Reproduce
To compare streamer performance between hudi-0.14 and hudi-1.0 we ran two
streamer applications in `--continuous` mode reading from the same kafka topic,
each reading 50 million events per batch starting at the same offsets (one hour
ago so that each would read the full 50M events per batch).
The configuration is the same except for two hudi-1x only properties:
`_hoodie.record.size.estimator.max.commits: 100`
`hoodie.write.table.version: 6`
```# Elided props for testperfhudi1tv6registryg
2025-09-25T16:16:13,908 INFO [delta-streamer]
org.apache.hudi.utilities.streamer.HoodieStreamer$StreamSyncService:Creating
Hudi Streamer with configs:
_hoodie.record.size.estimator.max.commits: 100
auto.offset.reset: latest
bootstrap.servers: *** elided ***
client.rack: use2-az1
com.secureworks.spark.hudi.metrics.kafka.enabled: true
com.secureworks.spark.hudi.metrics.kafka.topic: *** elided ***
com.secureworks.spark.hudi.metrics.statsd.hostname: *** elided ***
group.id: *** elided ***
hoodie.archive.async: false
hoodie.archive.automatic: true
hoodie.auto.adjust.lock.configs: true
hoodie.base.path: *** elided ***
hoodie.clean.async: false
hoodie.cleaner.hours.retained: 4
hoodie.cleaner.parallelism: 600
hoodie.cleaner.policy: KEEP_LATEST_BY_HOURS
hoodie.cleaner.policy.failed.writes: LAZY
hoodie.clustering.async.enabled: false
hoodie.copyonwrite.insert.auto.split: true
hoodie.datasource.fetch.table.enable: true
hoodie.datasource.write.hive_style_partitioning: true
hoodie.datasource.write.keygenerator.class:
org.apache.hudi.keygen.CustomKeyGenerator
hoodie.datasource.write.operation: insert
hoodie.datasource.write.partitioner.class:
com.secureworks.spark.hudi.protobuf.helpers.partitioner.ResourceIdPartitioner
hoodie.datasource.write.partitionpath.field: tenant:SIMPLE,date:SIMPLE
hoodie.datasource.write.precombine.field: event_time_usec
hoodie.datasource.write.reconcile.schema: false
hoodie.datasource.write.recordkey.field: resource_id
hoodie.fail.on.timeline.archiving: false
hoodie.filesystem.view.incr.timeline.sync.enable: true
hoodie.filesystem.view.remote.timeout.secs: 30
hoodie.keep.max.commits: 1080
hoodie.keep.min.commits: 1050
hoodie.merge.allow.duplicate.on.inserts: true
hoodie.metadata.enable: false
hoodie.metrics.datadog.api.key: *** elided ***
hoodie.metrics.datadog.api.site: US
hoodie.metrics.datadog.metric.host: api.datadoghq
hoodie.metrics.datadog.metric.prefix: hudi.ingest
hoodie.metrics.datadog.metric.tags: *** elided ***
hoodie.metrics.on: true
hoodie.metrics.reporter.metricsname.prefix:
hoodie.metrics.reporter.prefix.tablename: false
hoodie.metrics.reporter.type: DATADOG
hoodie.parquet.compression.codec: zstd
hoodie.streamer.end.offsets: *** elided ***
hoodie.streamer.kafka.source.maxEvents: 50000000
hoodie.streamer.schemaprovider.registry.url: *** elided ***
hoodie.streamer.source.kafka.enable.commit.offset: true
hoodie.streamer.source.kafka.minPartitions: 675
hoodie.streamer.source.kafka.topic: *** elided ***
hoodie.streamer.source.schema.subject: *** elided ***
hoodie.table.name: testperfhudi1tv6registryg
hoodie.table.partition.fields: tenant,date
hoodie.table.type: MERGE_ON_READ
hoodie.write.auto.upgrade: false
hoodie.write.concurrency.mode: OPTIMISTIC_CONCURRENCY_CONTROL
hoodie.write.lock.dynamodb.billing_mode: PROVISIONED
hoodie.write.lock.dynamodb.endpoint_url:
https://dynamodb.us-east-2.amazonaws.com/
hoodie.write.lock.dynamodb.partition_key: testperfhudi1tv6registryg
hoodie.write.lock.dynamodb.region: us-east-2
hoodie.write.lock.dynamodb.table: HudiLocker
hoodie.write.lock.provider:
org.apache.hudi.aws.transaction.lock.DynamoDBBasedLockProvider
hoodie.write.table.version: 6
```
diff w/ 0.14:
```
% diff /tmp/testperfhudi014registryg-elided-props.txt
/tmp/testperfhudi1tv6registryg-elided-props.txt
1,2c1,3
< # Elided props for testperfhudi014registryg
< 2025-09-25T15:19:03,161 INFO [delta-streamer]
org.apache.hudi.utilities.streamer.HoodieStreamer$StreamSyncService:Creating
Hudi Streamer with configs:
---
> # Elided props for testperfhudi1tv6registryg
> 2025-09-25T16:16:13,908 INFO [delta-streamer]
org.apache.hudi.utilities.streamer.HoodieStreamer$StreamSyncService:Creating
Hudi Streamer with configs:
> _hoodie.record.size.estimator.max.commits: 100
5c6
< client.rack: use2-az2
---
> client.rack: use2-az1
54c55
< hoodie.table.name: testperfhudi014registryg
---
> hoodie.table.name: testperfhudi1tv6registryg
61c62
< hoodie.write.lock.dynamodb.partition_key: testperfhudi014registryg
---
> hoodie.write.lock.dynamodb.partition_key: testperfhudi1tv6registryg
64a66
> hoodie.write.table.version: 6
```
### Expected behavior
Hudi-1.x does not produce significantly more files and file groups than
hudi-0.14 on the same input data
### Environment Description
* Hudi version: 1.0.2 vs 0.14.1
* Spark version: 3.5.4 vs 3.4.3
* Hadoop version: 3.4.1
* Storage (HDFS/S3/GCS..): S3
* Running on Docker? (yes/no): k8s
### Additional context
_No response_
### Stacktrace
```shell
```
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]