Hi Jian, Thanks for the KIP. This looks like a sensible idea to improve the default configuration for this topic.
Thanks, Andrew > On 14 Nov 2025, at 05:27, jian fu <[email protected]> wrote: > > Just bumping this thread for more discussion. > > Thanks! > > jian fu <[email protected]> 于2025年11月5日周三 14:54写道: > >> Hi everyone, >> >> I'd like to start a discussion on KIP-1235, a small change to set >> min.isr=2 for the __remote_log_metadata topic. >> >> KIP: >> https://cwiki.apache.org/confluence/display/KAFKA/KIP-1235%3A+Correct+the+default+min.insync.replicas+to+2+for+the+__remote_log_metadata+topic >> >> Problem: >> >> Currently, __remote_log_metadata relies on the broker default for min.isr >> (typically 1). >> With min.isr=1 and factor=3, acknowledged writes can be lost if the leader >> crashes before replication >> completes, making remote data permanently unreachable. >> >> What's more, the similar topic: __transaction_state sets min.isr=2 via >> transaction.state.log.min.isr. >> We should provide the same durability for __remote_log_metadata. >> >> >> Proposed Change: >> >> Add one new broker config: remote.log.metadata.topic.min.isr (default: 2) >> >> When creating __remote_log_metadata, TopicBasedRemoteLogMetadataManager >> will set >> min.insync.replicas=2 at the topic level. The change only affects new >> clusters enabling Tiered Storage. Existing topics are unchanged. >> >> >> Looking forward to your feedback! >> >> Best regards, >> Jian >>
