Hi Lijun, RemoteLogSegmentUpdateMetadata event does not have all the fields/attributes similar to RemoteLogSegmentMetadata event.
Assume that after compaction, for a segment, we have only COPY_SEGMENT_FINISHED records. How do you plan to retrieve the other fields after broker restart? Thanks, Kamal On Mon, Mar 30, 2026, 23:22 Lijun Tong <[email protected]> wrote: > Hi Kamal, > > Thanks for taking another look at the KIP. > 1. I have removed the left-over line about using another new topic from the > KIP. > 2. > > > 2. Assume that the topic is enabled with compaction and only one event is > > maintained per segment. If there is a transient error in the remote log > > deletion, > > then the COPY_SEGMENT started / finished events might be compacted by > > the DELETE_SEGMENT_STARTED events. If the broker is restarted during > > this time, will there be dangling remote log segments? Currently, > > during restart, the broker discards the events if it does not see the > > COPY_SEGMENT_STARTED events. > > > I am glad you asked this question, I didn't mention this part in my current > design to avoid distractions from the main design, but I plan to add > another background thread to clean up all the stale messages by comparing > the message's endOffset with the topic partition's log start offset. I > believe this would help remove all the dangling messages. > > Thanks, > Lijun TOng > > Kamal Chandraprakash <[email protected]> 于2026年3月29日周日 > 22:48写道: > > > Hi Lijun, > > > > Sorry for the late reply. Went over the KIP again. Overall LGTM. Few > > points: > > > > > This KIP aims to solve this issue through introducing another compacted > > topic for the brokers to bootstrap the state from > > > > 1. Shall we update the motivation section to mention that another topic > is > > not introduced? > > 2. Assume that the topic is enabled with compaction and only one event is > > maintained per segment. If there is a transient error in the remote log > > deletion, > > then the COPY_SEGMENT started / finished events might be compacted by > > the DELETE_SEGMENT_STARTED events. If the broker is restarted during > > this time, will there be dangling remote log segments? Currently, > > during restart, the broker discards the events if it does not see the > > COPY_SEGMENT_STARTED events. > > > > Thanks, > > Kamal > > > > On Thu, Mar 26, 2026 at 5:08 AM Lijun Tong <[email protected]> > > wrote: > > > > > Hi, > > > > > > I have started a Vote thread for this KIP, considering all questions > > raised > > > so far have been answered. I am happy to continue the discussion if > > needed, > > > otherwise, this is a friendly reminder on the vote for this KIP. > > > > > > Thanks, > > > Lijun Tong > > > > > > > > > > > > Lijun Tong <[email protected]> 于2026年1月19日周一 17:59写道: > > > > > > > Hey Kamal, > > > > > > > > Thanks for raising these questions. Here are my responses to your > > > > questions: > > > > Q1 and Q2: > > > > I think both questions boil down to how to release this new feature, > > both > > > > questions are valid concerns. The solution I have in mind is this > > feature > > > > is *gated by the metadata version*. The new tombstone semantics and > the > > > > additional fields (for example in RemoteLogSegmentUpdateRecord) are > > only > > > > enabled once the cluster metadata version is upgraded to the version > > that > > > > introduces this feature. As long as the cluster metadata version is > not > > > > bumped, the system will not produce tombstone records. Therefore, > > during > > > > rolling upgrades (mixed 4.2/4.3 brokers), the feature remains > > effectively > > > > disabled. Tombstones will only start being produced after the > metadata > > > > version is upgraded, at which point all brokers are already required > to > > > > support the new behavior. > > > > > > > > Since Kafka does not support metadata version downgrades at the > moment, > > > > once a metadata version that supports this feature is enabled, it > > cannot > > > be > > > > downgraded to a version that does not support it. I will add these > > > details > > > > to the KIP later. > > > > Q3. This is an *editing mistake* in the KIP. Thanks for pointing it > > out — > > > > the enum value has already been corrected in the latest revision to > > > remove > > > > the unused placeholder and keep the state values contiguous and > > > consistent. > > > > Q4. I don't foresee the quota mechanism will interfere with the state > > > > transition in any way so far, let me know if any concern hits you. > > > > > > > > Thanks, > > > > Lijun > > > > > > > > Kamal Chandraprakash <[email protected]> 于2026年1月18日周日 > > > > 00:40写道: > > > > > > > >> Hi Lijun, > > > >> > > > >> Thanks for updating the KIP! > > > >> > > > >> The updated migration plan looks clean to me. Few questions: > > > >> > > > >> 1. The ConsumerTask in 4.2 Kafka build does not handle the tombstone > > > >> records. Should the tombstone records be sent only when all the > > brokers > > > >> are > > > >> upgraded to 4.3 version? > > > >> > > > >> 2. Once all the brokers are upgraded and the __remote_log_metadata > > topic > > > >> cleanup policy changed to compact. Then, downgrading the brokers is > > not > > > >> allowed as the records without key will throw an error while > producing > > > the > > > >> compacted topic. Shall we mention this in the compatibility section? > > > >> > > > >> 3. In the RemoteLogSegmentState Enum, why is the value 1 marked as > > > unused? > > > >> > > > >> 4. Regarding the key (TopicIdPartition:EndOffset:BrokerLeaderEpoch), > > we > > > >> may > > > >> have to check for scenarios where there is segment lag due to remote > > log > > > >> write quota. Will the state transition work correctly? Will come > back > > to > > > >> this later. > > > >> > > > >> Thanks, > > > >> Kamal > > > >> > > > >> On Fri, Jan 16, 2026 at 4:50 AM jian fu <[email protected]> > wrote: > > > >> > > > >> > Hi Lijun and Kamal > > > >> > I also think we don't need to keep delJIanpolicy in final > config,if > > > >> so,we > > > >> > should always keep remembering all of our topic retention time > must > > > less > > > >> > than the value,right?It is one protect with risk involved. > > > >> > Regards > > > >> > JIan > > > >> > > > > >> > > > > >> > > > > >> > Lijun Tong <[email protected]>于2026年1月16日 周五06:45写道: > > > >> > > > > >> > > Hey Kamal, > > > >> > > > > > >> > > Some additional points about the Q4, > > > >> > > > > > >> > > > The user can decide when to change their internal topic > cleanup > > > >> policy > > > >> > to > > > >> > > > compact. If someone retains > > > >> > > > the data in the remote storage for 3 months, then they can > > migrate > > > >> to > > > >> > the > > > >> > > > compacted topic after 3 months > > > >> > > > post rolling out this change. And, update their cleanup policy > > to > > > >> > > [compact, > > > >> > > > delete]. > > > >> > > > > > >> > > > > > >> > > I don't think it's a good idea to keep delete in the final > cleanup > > > >> policy > > > >> > > for the topic `__remote_log_metadata`, as this still requires > the > > > >> user to > > > >> > > keep track of the max retention hours of topics that have remote > > > >> storage > > > >> > > enabled, and it's operational burden. It's also hard to reason > > about > > > >> what > > > >> > > will happen if the user configures the wrong retention.ms. I > hope > > > >> this > > > >> > > makes sense. > > > >> > > > > > >> > > > > > >> > > Thanks, > > > >> > > Lijun Tong > > > >> > > > > > >> > > Lijun Tong <[email protected]> 于2026年1月15日周四 11:43写道: > > > >> > > > > > >> > > > Hey Kamal, > > > >> > > > > > > >> > > > Thanks for your reply! I am glad we are on the same page with > > > making > > > >> > the > > > >> > > > __remote_log_metadata topic compacted optional for the user > > now, I > > > >> will > > > >> > > > update the KIP with this change. > > > >> > > > > > > >> > > > For the Q2: > > > >> > > > With the key designed as > > > >> TopicId:Partition:EndOffset:BrokerLeaderEpoch, > > > >> > > > even the same broker retries the upload multiple times for the > > > same > > > >> log > > > >> > > > segment, the latest retry attempt with the latest segment UUID > > > will > > > >> > > > overwrite the previous attempts' value since they share the > same > > > >> key, > > > >> > so > > > >> > > we > > > >> > > > don't need to explicitly track the failed upload metadata, > > because > > > >> it's > > > >> > > > gone already by the later attempt. That's my understanding > about > > > the > > > >> > > > RLMCopyTask, correct me if I am wrong. > > > >> > > > > > > >> > > > Thanks, > > > >> > > > Lijun Tong > > > >> > > > > > > >> > > > Kamal Chandraprakash <[email protected]> > > > 于2026年1月14日周三 > > > >> > > > 21:18写道: > > > >> > > > > > > >> > > >> Hi Lijun, > > > >> > > >> > > > >> > > >> Thanks for the reply! > > > >> > > >> > > > >> > > >> Q1: Sounds good. Could you clarify it in the KIP that the > same > > > >> > > partitioner > > > >> > > >> will be used? > > > >> > > >> > > > >> > > >> Q2: With TopicId:Partition:EndOffset:BrokerLeaderEpoch key, > if > > > the > > > >> > same > > > >> > > >> broker retries the upload due to intermittent > > > >> > > >> issues in object storage (or) RLMM. Then, those failed upload > > > >> metadata > > > >> > > >> also > > > >> > > >> need to be cleared. > > > >> > > >> > > > >> > > >> Q3: We may have to skip the null value records in the > > > ConsumerTask. > > > >> > > >> > > > >> > > >> Q4a: The idea is to keep the cleanup policy as "delete" and > > also > > > >> send > > > >> > > the > > > >> > > >> tombstone markers > > > >> > > >> to the existing `__remote_log_metadata` topic. And, handle > the > > > >> > tombstone > > > >> > > >> records in the ConsumerTask. > > > >> > > >> > > > >> > > >> The user can decide when to change their internal topic > cleanup > > > >> policy > > > >> > > to > > > >> > > >> compact. If someone retains > > > >> > > >> the data in the remote storage for 3 months, then they can > > > migrate > > > >> to > > > >> > > the > > > >> > > >> compacted topic after 3 months > > > >> > > >> post rolling out this change. And, update their cleanup > policy > > to > > > >> > > >> [compact, > > > >> > > >> delete]. > > > >> > > >> > > > >> > > >> Thanks, > > > >> > > >> Kamal > > > >> > > >> > > > >> > > >> On Thu, Jan 15, 2026 at 4:12 AM Lijun Tong < > > > >> [email protected]> > > > >> > > >> wrote: > > > >> > > >> > > > >> > > >> > Hey Jian, > > > >> > > >> > > > > >> > > >> > Thanks for your time to review this KIP. I appreciate that > > you > > > >> > > propose a > > > >> > > >> > simpler migration solution to onboard the new feature. > > > >> > > >> > > > > >> > > >> > There are 2 points that I think can be further refined on: > > > >> > > >> > > > > >> > > >> > 1). make the topic compacted optional, although the new > > feature > > > >> will > > > >> > > >> > continue to emit tombstone message for those expired log > > > segments > > > >> > even > > > >> > > >> when > > > >> > > >> > the topic is still on time-based retention mode, so once > user > > > >> > switched > > > >> > > >> to > > > >> > > >> > using the compacted topic, those expired messages can still > > be > > > >> > deleted > > > >> > > >> > despite the topic is not retention based anymore. > > > >> > > >> > 2). we need to expose some flag to the user to indicate > > whether > > > >> the > > > >> > > >> topic > > > >> > > >> > can be flipped to compacted by checking whether all the old > > > >> format > > > >> > > >> > keyed-less message has expired, and allow user to choose to > > > flip > > > >> to > > > >> > > >> > compacted only when the flag is true. > > > >> > > >> > > > > >> > > >> > Thanks for sharing your idea. I will update the KIP later > > with > > > >> this > > > >> > > new > > > >> > > >> > idea. > > > >> > > >> > > > > >> > > >> > Best, > > > >> > > >> > Lijun Tong > > > >> > > >> > > > > >> > > >> > > > > >> > > >> > jian fu <[email protected]> 于2026年1月12日周一 04:55写道: > > > >> > > >> > > > > >> > > >> > > Hi Lijun Tong: > > > >> > > >> > > > > > >> > > >> > > Thanks for your KIP which raise this critical issue. > > > >> > > >> > > > > > >> > > >> > > what about just keep one topic instead of involve another > > > >> topic. > > > >> > > >> > > for existed topic data's migration. maybe we can use this > > way > > > >> to > > > >> > > solve > > > >> > > >> > the > > > >> > > >> > > issue: > > > >> > > >> > > (1) set the retention date > all of topic which enable > > remote > > > >> > > >> storage's > > > >> > > >> > > retention time > > > >> > > >> > > (2) deploy new kafka version with feature: which send > the > > > >> message > > > >> > > >> with > > > >> > > >> > key > > > >> > > >> > > (3) wait all the message expired and new message with key > > > >> coming > > > >> > to > > > >> > > >> the > > > >> > > >> > > topic > > > >> > > >> > > (4) convert the topic to compact > > > >> > > >> > > > > > >> > > >> > > I don't test it. Just propose this solution according to > > code > > > >> > review > > > >> > > >> > > result. just for your reference. > > > >> > > >> > > The steps maybe a little complex. but it can avoiding add > > new > > > >> > topic. > > > >> > > >> > > > > > >> > > >> > > Regards > > > >> > > >> > > Jian > > > >> > > >> > > > > > >> > > >> > > Lijun Tong <[email protected]> 于2026年1月8日周四 > 09:17写道: > > > >> > > >> > > > > > >> > > >> > > > Hey Kamal, > > > >> > > >> > > > > > > >> > > >> > > > > > > >> > > >> > > > Thanks for your time for the review. > > > >> > > >> > > > > > > >> > > >> > > > > > > >> > > >> > > > Here is my response to your questions: > > > >> > > >> > > > > > > >> > > >> > > > Q1 At this point, I don’t see a need to change > > > >> > > >> > > > RemoteLogMetadataTopicPartitioner for this design. > > Nothing > > > in > > > >> > the > > > >> > > >> > current > > > >> > > >> > > > approach appears to require a partitioner change, but > I’m > > > >> open > > > >> > to > > > >> > > >> > > > revisiting if a concrete need arises. > > > >> > > >> > > > > > > >> > > >> > > > Q2 I have some reservations about using SegmentId:State > > as > > > >> the > > > >> > > key. > > > >> > > >> A > > > >> > > >> > > > practical challenge we see today is that the same > logical > > > >> > segment > > > >> > > >> can > > > >> > > >> > be > > > >> > > >> > > > retried multiple times with different SegmentIds across > > > >> brokers. > > > >> > > If > > > >> > > >> the > > > >> > > >> > > key > > > >> > > >> > > > is SegmentId-based, it becomes harder to discover and > > > >> tombstone > > > >> > > all > > > >> > > >> > > related > > > >> > > >> > > > attempts when the segment eventually expires. The > > > >> > > >> > > > TopicId:Partition:EndOffset:BrokerLeaderEpoch key is > > > >> > deterministic > > > >> > > >> for > > > >> > > >> > a > > > >> > > >> > > > logical segment attempt and helps group retries by > epoch, > > > >> which > > > >> > > >> > > simplifies > > > >> > > >> > > > cleanup and reasoning about state. I’d love to > understand > > > the > > > >> > > >> benefits > > > >> > > >> > > > you’re seeing with SegmentId:State compared to the > > > >> > > >> offset/epoch-based > > > >> > > >> > key > > > >> > > >> > > > so we can weigh the trade-offs. > > > >> > > >> > > > > > > >> > > >> > > > On partitioning: with this proposal, all states for a > > given > > > >> user > > > >> > > >> > > > topic-partition still map to the same metadata > partition. > > > >> That > > > >> > > >> remains > > > >> > > >> > > true > > > >> > > >> > > > for the existing __remote_log_metadata (unchanged > > > >> partitioner) > > > >> > and > > > >> > > >> for > > > >> > > >> > > the > > > >> > > >> > > > new __remote_log_metadata_compacted, preserving the > > > >> properties > > > >> > > >> > > > RemoteMetadataCache relies on. > > > >> > > >> > > > > > > >> > > >> > > > Q3 It should be fine for ConsumerTask to ignore > tombstone > > > >> > records > > > >> > > >> (null > > > >> > > >> > > > values) and no-op. > > > >> > > >> > > > > > > >> > > >> > > > Q4 Although TBRLMM is a sample RLMM implementation, > it’s > > > >> > currently > > > >> > > >> the > > > >> > > >> > > only > > > >> > > >> > > > OSS option and is widely used. The new > > > >> > > >> __remote_log_metadata_compacted > > > >> > > >> > > > topic offers clear operational benefits in that > context. > > We > > > >> can > > > >> > > also > > > >> > > >> > > > provide a configuration to let users choose whether > they > > > >> want to > > > >> > > >> keep > > > >> > > >> > the > > > >> > > >> > > > audit topic (__remote_log_metadata) in their cluster. > > > >> > > >> > > > > > > >> > > >> > > > Q4a Enabling compaction on __remote_log_metadata alone > > may > > > >> not > > > >> > > fully > > > >> > > >> > > > address the unbounded growth, since we also need to > emit > > > >> > > tombstones > > > >> > > >> for > > > >> > > >> > > > expired keys to delete them. Deferring compaction and > > > >> > tombstoning > > > >> > > to > > > >> > > >> > user > > > >> > > >> > > > configuration could make the code flow complicated, > also > > > add > > > >> > > >> > operational > > > >> > > >> > > > complexity and make outcomes less predictable. The > > proposal > > > >> aims > > > >> > > to > > > >> > > >> > > provide > > > >> > > >> > > > a consistent experience by defining deterministic keys > > and > > > >> > > emitting > > > >> > > >> > > > tombstones as part of the broker’s responsibilities, > > while > > > >> still > > > >> > > >> > allowing > > > >> > > >> > > > users to opt out of the audit topic if they prefer. > But I > > > am > > > >> > open > > > >> > > to > > > >> > > >> > more > > > >> > > >> > > > discussion if there is any concrete need I don't > foresee. > > > >> > > >> > > > > > > >> > > >> > > > > > > >> > > >> > > > Thanks, > > > >> > > >> > > > > > > >> > > >> > > > Lijun Tong > > > >> > > >> > > > > > > >> > > >> > > > Kamal Chandraprakash <[email protected]> > > > >> > > 于2026年1月6日周二 > > > >> > > >> > > > 01:01写道: > > > >> > > >> > > > > > > >> > > >> > > > > Hi Lijun, > > > >> > > >> > > > > > > > >> > > >> > > > > Thanks for the KIP! Went over the first pass. > > > >> > > >> > > > > > > > >> > > >> > > > > Few Questions: > > > >> > > >> > > > > > > > >> > > >> > > > > 1. Are we going to maintain the same > > > >> > > >> > RemoteLogMetadataTopicPartitioner > > > >> > > >> > > > > < > > > >> > > >> > > > > > > > >> > > >> > > > > > > >> > > >> > > > > > >> > > >> > > > > >> > > >> > > > >> > > > > > >> > > > > >> > > > > > > https://sourcegraph.com/github.com/apache/kafka/-/blob/storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/RemoteLogMetadataTopicPartitioner.java > > > >> > > >> > > > > > > > > >> > > >> > > > > for both the topics? It is not clear in the KIP, > could > > > you > > > >> > > clarify > > > >> > > >> > it? > > > >> > > >> > > > > 2. Can the key be changed to SegmentId:State instead > of > > > >> > > >> > > > > TopicId:Partition:EndOffset:BrokerLeaderEpoch if the > > same > > > >> > > >> partitioner > > > >> > > >> > > is > > > >> > > >> > > > > used? It is good to maintain all the segment states > > for a > > > >> > > >> > > > > user-topic-partition in the same metadata partition. > > > >> > > >> > > > > 3. Should we have to handle the records with null > value > > > >> > > >> (tombstone) > > > >> > > >> > in > > > >> > > >> > > > the > > > >> > > >> > > > > ConsumerTask > > > >> > > >> > > > > < > > > >> > > >> > > > > > > > >> > > >> > > > > > > >> > > >> > > > > > >> > > >> > > > > >> > > >> > > > >> > > > > > >> > > > > >> > > > > > > https://sourcegraph.com/github.com/apache/kafka/-/blob/storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/ConsumerTask.java?L166 > > > >> > > >> > > > > > > > > >> > > >> > > > > ? > > > >> > > >> > > > > 4. TBRLMM > > > >> > > >> > > > > < > > > >> > > >> > > > > > > > >> > > >> > > > > > > >> > > >> > > > > > >> > > >> > > > > >> > > >> > > > >> > > > > > >> > > > > >> > > > > > > https://sourcegraph.com/github.com/apache/kafka/-/blob/storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/TopicBasedRemoteLogMetadataManager.java > > > >> > > >> > > > > > > > > >> > > >> > > > > is a sample plugin implementation of RLMM. Not sure > > > whether > > > >> > the > > > >> > > >> > > community > > > >> > > >> > > > > will agree to add one more internal topic for this > > plugin > > > >> > impl. > > > >> > > >> > > > > 4a. Can we modify the new messages to the > > > >> > __remote_log_metadata > > > >> > > >> topic > > > >> > > >> > > to > > > >> > > >> > > > > contain the key and leave it to the user to enable > > > >> compaction > > > >> > > for > > > >> > > >> > this > > > >> > > >> > > > > topic if they need? > > > >> > > >> > > > > > > > >> > > >> > > > > Thanks, > > > >> > > >> > > > > Kamal > > > >> > > >> > > > > > > > >> > > >> > > > > On Tue, Jan 6, 2026 at 7:35 AM Lijun Tong < > > > >> > > >> [email protected]> > > > >> > > >> > > > wrote: > > > >> > > >> > > > > > > > >> > > >> > > > > > Hey Henry, > > > >> > > >> > > > > > > > > >> > > >> > > > > > Thank you for your time and response! I really like > > > your > > > >> > > >> KIP-1248 > > > >> > > >> > > about > > > >> > > >> > > > > > offloading the consumption of remote log away from > > the > > > >> > broker, > > > >> > > >> and > > > >> > > >> > I > > > >> > > >> > > > > think > > > >> > > >> > > > > > with that change, the topic that enables the tiered > > > >> storage > > > >> > > can > > > >> > > >> > also > > > >> > > >> > > > have > > > >> > > >> > > > > > longer retention configurations and would benefit > > from > > > >> this > > > >> > > KIP > > > >> > > >> > too. > > > >> > > >> > > > > > > > > >> > > >> > > > > > Some suggestions: In your example scenarios, it > would > > > >> also > > > >> > be > > > >> > > >> good > > > >> > > >> > to > > > >> > > >> > > > add > > > >> > > >> > > > > > > an example of remote log segment deletion > triggered > > > by > > > >> > > >> retention > > > >> > > >> > > > policy > > > >> > > >> > > > > > > which will trigger generation of tombstone event > > into > > > >> > > metadata > > > >> > > >> > > topic > > > >> > > >> > > > > and > > > >> > > >> > > > > > > trigger log compaction/deletion 24 hour later, I > > > think > > > >> > this > > > >> > > is > > > >> > > >> > the > > > >> > > >> > > > key > > > >> > > >> > > > > > > event to cap the metadata topic size. > > > >> > > >> > > > > > > > > >> > > >> > > > > > > > > >> > > >> > > > > > Regarding to this suggestion, I am not sure whether > > > >> > Scenario 4 > > > >> > > >> > > > > > < > > > >> > > >> > > > > > > > > >> > > >> > > > > > > > >> > > >> > > > > > > >> > > >> > > > > > >> > > >> > > > > >> > > >> > > > >> > > > > > >> > > > > >> > > > > > > https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=406618613#KIP1266:BoundingTheNumberOfRemoteLogMetadataMessagesviaCompactedTopic-Scenario4:SegmentDeletion > > > >> > > >> > > > > > > > > > >> > > >> > > > > > has > > > >> > > >> > > > > > covered it. I can add more rows in the Timeline > Table > > > >> like > > > >> > > >> > T5+24hour > > > >> > > >> > > to > > > >> > > >> > > > > > indicate the messages are gone by then to > explicitly > > > show > > > >> > that > > > >> > > >> > > messages > > > >> > > >> > > > > are > > > >> > > >> > > > > > deleted, thus the number of messages are capped in > > the > > > >> > topic. > > > >> > > >> > > > > > > > > >> > > >> > > > > > Regarding whether the topic __remote_log_metadata > is > > > >> still > > > >> > > >> > > necessary, I > > > >> > > >> > > > > am > > > >> > > >> > > > > > inclined to continue to have this topic at least > for > > > >> > debugging > > > >> > > >> > > purposes > > > >> > > >> > > > > so > > > >> > > >> > > > > > we can build confidence about the compacted topic > > > >> change, we > > > >> > > can > > > >> > > >> > > > > > always choose to remove this topic in the future > once > > > we > > > >> all > > > >> > > >> agree > > > >> > > >> > it > > > >> > > >> > > > > > provides limited value for the users. > > > >> > > >> > > > > > > > > >> > > >> > > > > > Thanks, > > > >> > > >> > > > > > Lijun Tong > > > >> > > >> > > > > > > > > >> > > >> > > > > > > > > >> > > >> > > > > > Henry Haiying Cai via dev <[email protected]> > > > >> > 于2026年1月5日周一 > > > >> > > >> > > 16:19写道: > > > >> > > >> > > > > > > > > >> > > >> > > > > > > Lijun, > > > >> > > >> > > > > > > > > > >> > > >> > > > > > > Thanks for the proposal and I liked your idea of > > > using > > > >> a > > > >> > > >> > compacted > > > >> > > >> > > > > topic > > > >> > > >> > > > > > > for tiered storage metadata topic. > > > >> > > >> > > > > > > > > > >> > > >> > > > > > > In our setup, we have set a shorter retention (3 > > > days) > > > >> for > > > >> > > the > > > >> > > >> > > tiered > > > >> > > >> > > > > > > storage metadata topic to control the size > growth. > > > We > > > >> can > > > >> > > do > > > >> > > >> > that > > > >> > > >> > > > > since > > > >> > > >> > > > > > we > > > >> > > >> > > > > > > control all topic's retention policy in our > > clusters > > > >> and > > > >> > we > > > >> > > >> set a > > > >> > > >> > > > > uniform > > > >> > > >> > > > > > > retention.policy for all our tiered storage > topics. > > > I > > > >> can > > > >> > > see > > > >> > > >> > > other > > > >> > > >> > > > > > > users/companies will not be able to enforce that > > > >> retention > > > >> > > >> policy > > > >> > > >> > > to > > > >> > > >> > > > > all > > > >> > > >> > > > > > > tiered storage topics. > > > >> > > >> > > > > > > > > > >> > > >> > > > > > > Some suggestions: In your example scenarios, it > > would > > > >> also > > > >> > > be > > > >> > > >> > good > > > >> > > >> > > to > > > >> > > >> > > > > add > > > >> > > >> > > > > > > an example of remote log segment deletion > triggered > > > by > > > >> > > >> retention > > > >> > > >> > > > policy > > > >> > > >> > > > > > > which will trigger generation of tombstone event > > into > > > >> > > metadata > > > >> > > >> > > topic > > > >> > > >> > > > > and > > > >> > > >> > > > > > > trigger log compaction/deletion 24 hour later, I > > > think > > > >> > this > > > >> > > is > > > >> > > >> > the > > > >> > > >> > > > key > > > >> > > >> > > > > > > event to cap the metadata topic size. > > > >> > > >> > > > > > > > > > >> > > >> > > > > > > For the original unbounded remote_log_metadata > > topic, > > > >> I am > > > >> > > not > > > >> > > >> > sure > > > >> > > >> > > > > > > whether we still need it or not. If it is left > > only > > > >> for > > > >> > > audit > > > >> > > >> > > trail > > > >> > > >> > > > > > > purpose, people can set up a data ingestion > > pipeline > > > to > > > >> > > ingest > > > >> > > >> > the > > > >> > > >> > > > > > content > > > >> > > >> > > > > > > of metadata topic into a separate storage > location. > > > I > > > >> > think > > > >> > > >> we > > > >> > > >> > can > > > >> > > >> > > > > have > > > >> > > >> > > > > > a > > > >> > > >> > > > > > > flag to have only one metadata topic (the > compacted > > > >> > > version). > > > >> > > >> > > > > > > > > > >> > > >> > > > > > > > > > >> > > >> > > > > > > On Monday, January 5, 2026 at 01:22:42 PM PST, > > Lijun > > > >> Tong > > > >> > < > > > >> > > >> > > > > > > [email protected]> wrote: > > > >> > > >> > > > > > > > > > >> > > >> > > > > > > > > > >> > > >> > > > > > > > > > >> > > >> > > > > > > > > > >> > > >> > > > > > > > > > >> > > >> > > > > > > Hello Kafka Community, > > > >> > > >> > > > > > > > > > >> > > >> > > > > > > I would like to start a discussion on KIP-1266, > > which > > > >> > > >> proposes to > > > >> > > >> > > add > > > >> > > >> > > > > > > another new compacted remote log metadata topic > for > > > the > > > >> > > tiered > > > >> > > >> > > > storage, > > > >> > > >> > > > > > to > > > >> > > >> > > > > > > limit the number of messages that need to be > > iterated > > > >> to > > > >> > > build > > > >> > > >> > the > > > >> > > >> > > > > remote > > > >> > > >> > > > > > > metadata state. > > > >> > > >> > > > > > > > > > >> > > >> > > > > > > KIP link: KIP-1266 Bounding The Number Of > > > >> > RemoteLogMetadata > > > >> > > >> > > Messages > > > >> > > >> > > > > via > > > >> > > >> > > > > > > Compacted RemoteLogMetadata Topic > > > >> > > >> > > > > > > < > > > >> > > >> > > > > > > > > > >> > > >> > > > > > > > > >> > > >> > > > > > > > >> > > >> > > > > > > >> > > >> > > > > > >> > > >> > > > > >> > > >> > > > >> > > > > > >> > > > > >> > > > > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-1266%3A+Bounding+The+Number+Of+RemoteLogMetadata+Messages+via+Compacted+Topic > > > >> > > >> > > > > > > > > > > >> > > >> > > > > > > > > > >> > > >> > > > > > > Background: > > > >> > > >> > > > > > > The current Tiered Storage implementation uses a > > > >> > > >> > > > __remote_log_metadata > > > >> > > >> > > > > > > topic with infinite retention and delete-based > > > cleanup > > > >> > > policy, > > > >> > > >> > > > causing > > > >> > > >> > > > > > > unbounded growth, slow broker bootstrap, no > > mechanism > > > >> to > > > >> > > >> clean up > > > >> > > >> > > > > expired > > > >> > > >> > > > > > > segment metadata, and inefficient re-reading from > > > >> offset 0 > > > >> > > >> during > > > >> > > >> > > > > > > leadership changes. > > > >> > > >> > > > > > > > > > >> > > >> > > > > > > Proposal: > > > >> > > >> > > > > > > A dual-topic approach that introduces a new > > > >> > > >> > > > > > __remote_log_metadata_compacted > > > >> > > >> > > > > > > topic using log compaction with deterministic > > > >> offset-based > > > >> > > >> keys, > > > >> > > >> > > > while > > > >> > > >> > > > > > > preserving the existing topic for audit history; > > this > > > >> > allows > > > >> > > >> > > brokers > > > >> > > >> > > > to > > > >> > > >> > > > > > > build their metadata cache exclusively from the > > > >> compacted > > > >> > > >> topic, > > > >> > > >> > > > > enables > > > >> > > >> > > > > > > cleanup of expired segment metadata through > > > tombstones, > > > >> > and > > > >> > > >> > > includes > > > >> > > >> > > > a > > > >> > > >> > > > > > > migration strategy to populate the new topic > during > > > >> > > >> > > > upgrade—delivering > > > >> > > >> > > > > > > bounded metadata growth and faster broker startup > > > while > > > >> > > >> > maintaining > > > >> > > >> > > > > > > backward compatibility. > > > >> > > >> > > > > > > > > > >> > > >> > > > > > > More details are in the attached KIP link. > > > >> > > >> > > > > > > Looking forward to your thoughts. > > > >> > > >> > > > > > > > > > >> > > >> > > > > > > Thank you for your time! > > > >> > > >> > > > > > > > > > >> > > >> > > > > > > Best, > > > >> > > >> > > > > > > Lijun Tong > > > >> > > >> > > > > > > > > > >> > > >> > > > > > > > > >> > > >> > > > > > > > >> > > >> > > > > > > >> > > >> > > > > > >> > > >> > > > > >> > > >> > > > >> > > > > > > >> > > > > > >> > > > > >> > > > > > > > > > >
