Hi Henry, LC1. OK, I see. About the optimization, I was also thinking we can upload the hashmap after each ground of first pass scan, so the hashmap contains the scan result of [A + B(will become A in future round)] at this stage. In the next round, we only need to read the [hashmap + new B] to build the updated hashmap. But that would not be able to build the blooming filter because of this optimization, right? Any thoughts?
LC2. Please add a section in the KIP to describe it. LC3. Agree. We should add a new configuration for it, right? LC5. Looks good. We should add them in the KIP. Thank you, Luke On Mon, May 18, 2026 at 3:31 PM Henry Haiying Cai <[email protected]> wrote: > Luke, > > Thanks for taking time to read the design and gave insightful critiques. > Here are the replies to your questions, if we align on the direction, we > can update the KIP for the new suggestions. > > LC1. We are not planning to upload the bloom filter to remote tired > storage, it is designed to be lazily built locally for the entries in the > dirty section B1 (the dirty section only in remote). This is considered to > be a smaller section comparing to the cleaned section A. On a long running > broker, the section A will become bigger and bigger with each compaction > cycle and B1 is relatively constant. The reason we decide not to upload > the bloom filter index is it is just one of the possible optimization data > structures. In the future people might come up with other better > probabilistic indexing data structure, we don't want to leave something > persistent and have to worry about migrating that in the future; > > LC2, for bandwidth control. We plan to use/extend the built-in throttler > in the LogCleaner. The compaction for remote log segment is going to > happen in chunks which provide a good place to throttle; > > LC3, Yes we can have a separate thread pool for Log Cleaners for tiered > storage topic since log cleaner for tiered storage runs longer; > > LC4, We are not going to have new methods in RemoteStorageManager for > uploading cleaner-offset-checkpoint. Instead, the > cleaner-offset-checkpoints are uploaded as part of the > RemoteLogSegmentMetadata (similar to how it includes segmentLeaderEpochs > map), so we are enhancing RemoteLogSegmentMetadata to include another map > for cleaner-offset-checkpoint for uploading/downloading. To access the > cleaner-offset-checkpoint map, we are adding a method in > RemoteLogMetadataManager: cleanerOffsetCheckpointForEpoch(int epoch), > similar to the existing highestOffsetForEpoch() method; > > LC5: Yes we can add metrics to monitor the log compaction for tiered > storage topics: the metrics to add would be compaction lag (dirty ratio for > tiered segments), compaction throughput (bytes/sec) time per compaction > cycle, and number of segments cleaned/uploaded. > >
