wombatu-kun opened a new pull request, #18231:
URL: https://github.com/apache/hudi/pull/18231
### Describe the issue this Pull Request addresses
When using Hudi as a lookup join dimension table in Flink, the entire table
is loaded into the cache on each reload. If dimension table is large - we got
OOM.
### Summary and Changelog
Introduced `LookupCache` — a minimal interface (addRow, getRows, clear,
close) that abstracts the cache backend from the lookup function.
`HeapLookupCache` — wraps the original HashMap behavior; default,
backward-compatible.
`RocksDBLookupCache` — off-heap implementation.
Rows are written to RocksDB one-by-one during reload (no intermediate
HashMap) so heap never spikes to O(n)
Each row gets its own RocksDB entry with a compound key <keyHex>_<counter>,
enabling efficient prefix-scan lookup
Flink's TypeSerializer<RowData> handles binary serialization; a
PassThroughSerializer stores the raw bytes without extra Java-serialization
overhead
clear() tears down and re-creates the RocksDBDAO (cleaning up the old temp
directory automatically)
Added two new config options to `FlinkOptions`:
`lookup.join.cache.type` = `heap` (default) | `rocksdb`
`lookup.join.rocksdb.path` = local directory path, default
`${java.io.tmpdir}/hudi-lookup-rocksdb`
`HoodieLookupFunction` refactored to use `LookupCache` interface.
Also fixes a pre-existing bug where currentCommit was never set after a
successful load, causing redundant full reloads on every TTL expiry
### Impact
Solved OutOfMemoryError on lookup joins with large Hudi tables.
### Risk Level
none
### Documentation Update
Need to add two new config options to documentation:
`lookup.join.cache.type` and `lookup.join.rocksdb.path`.
### Contributor's checklist
- [ ] Read through [contributor's
guide](https://hudi.apache.org/contribute/how-to-contribute)
- [ ] Enough context is provided in the sections above
- [ ] Adequate tests were added if applicable
--
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]