yihua opened a new pull request, #6113: URL: https://github.com/apache/hudi/pull/6113
## What is the purpose of the pull request This PR fixes the missing bloom filters in metadata table in the non-partitioned table due to incorrect record key generation. Before this PR, the file name is wrong when generating the metadata payload for the bloom filter. For example, below shows the file name used to construct the metadata payload: ``` Filename: 03656eb-c000-474b-945e-aa9298c3334d_1-0-1_0000001.parquet Bloom filter record key: DW/eaNVbRdo=xDmB/pnnQIMnCbUZywNZxw== Filename: f1a759f-8e00-4cc4-8af0-676d3c892657_1-0-1_0000002.parquet Bloom filter record key: DW/eaNVbRdo=t/6nT2vbZbsGoSkZBCOKZA== Filename: ca4aa60-2659-4fae-9d57-c4f51e8a7343_1-0-1_0000003.parquet Bloom filter record key: DW/eaNVbRdo=DsnvarlysKz9lJxfoZ81iA== ``` The file name misses the first character. In Bloom Index, when doing a lookup in the metadata table based on the actual file name, the corresponding bloom filter cannot be found because the record key generated during the lookup does not match what's stored in the metadata table, causing the upsert to fail: ``` BaseTableMetadata: BloomFilterIndex pair: 0f1a759f-8e00-4cc4-8af0-676d3c892657_1-0-1_0000002.parquet BaseTableMetadata: BloomFilterIndex pair: eca4aa60-2659-4fae-9d57-c4f51e8a7343_1-0-1_0000003.parquet ``` ``` Caused by: org.apache.hudi.exception.HoodieIndexException: Failed to get the bloom filter for (,0f1a759f-8e00-4cc4-8af0-676d3c892657_1-0-1_0000002.parquet) at org.apache.hudi.index.bloom.HoodieMetadataBloomIndexCheckFunction$BloomIndexLazyKeyCheckIterator.lambda$computeNext$2(HoodieMetadataBloomIndexCheckFunction.java:127) at java.util.HashMap.forEach(HashMap.java:1289) at org.apache.hudi.index.bloom.HoodieMetadataBloomIndexCheckFunction$BloomIndexLazyKeyCheckIterator.computeNext(HoodieMetadataBloomIndexCheckFunction.java:120) at org.apache.hudi.index.bloom.HoodieMetadataBloomIndexCheckFunction$BloomIndexLazyKeyCheckIterator.computeNext(HoodieMetadataBloomIndexCheckFunction.java:76) at org.apache.hudi.client.utils.LazyIterableIterator.next(LazyIterableIterator.java:119) ... 15 more ``` The fix is to generate the correct file name for the non-partitioned table. ## Brief change log - Fixes the logic of generating file name for the non-partitioned table in `HoodieTableMetadataUtil` - Adds unit tests for Bloom Index using metadata table, for both partitioned and non-partitioned table - Fixes commit metadata generation for non-partitioned table ## Verify this pull request This PR adds unit tests for Bloom Index using metadata table so that all existing tests run in two setups, w/ and w/o using metadata table for column stats and bloom filters. This PR also adds the tests for non-partitioned tables. Before the fix, the tests for non-partitioned tables fail. After the fix, the same set of tests succeeded. The fix is verified to resolve the problem for upserts on S3 using Bloom Index with metadata table read. ## Committer checklist - [ ] Has a corresponding JIRA in PR title & commit - [ ] Commit message is descriptive of the change - [ ] CI is green - [ ] Necessary doc changes done or have another open PR - [ ] For large changes, please consider breaking it into sub-tasks under an umbrella JIRA. -- 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: commits-unsubscr...@hudi.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org