Hi Andrew, Thanks for considering the suggestion and adding topic level configuration, which requires dead letter queue topics to have required configuration. This ensures that data topics are not mistakenly designated as DLQ topics.
However, there is still lack of protection on the writes for dlq topics where say 2 different applications app-T1 and app-T2 reading from T1 and T2 topics respectively, but both configure dlq.T1 as the dlq topic. This might not be an ideal behaviour expected by app-T1. Having said that, I also understand that it might not be an immediate concern and we should see how the usage goes for DLQ topics and can decide later if we need to solve this further by another KIP. Thanks for answering the questions and changes. I have no further questions. Regards, Apoorv Mittal On Fri, Feb 6, 2026 at 10:04 PM Andrew Schofield <[email protected]> wrote: > Hi Apoorv, > Thinking more about how to prevent arbitrary topics from being > accidentally or maliciously bombarded with DLQ records, I've added a > topic-level config which must be true in order for a topic to be used as a > DLQ topic for share groups. If a DLQ topic is auto-created, of course the > config will be set. > > Let me know what you think. > > Thanks, > Andrew > > On 2026/02/03 20:21:23 Andrew Schofield wrote: > > Hi Apoorv, > > Thanks for your question. It's an interesting one. > > > > I do not think it's a good idea to require a share consumer to need > WRITE permission to the group's DLQ topic. I do take your point about > unfortunate misconfigurations so I wonder if there's something else we > could do. Perhaps we could do an additional authorization check when a user > sets the errors.deadletterqueue.topic.name group config. > > > > Thanks, > > Andrew > > > > On 2026/02/02 21:20:10 Apoorv Mittal wrote: > > > Hi Andrew, > > > Thanks for the response and sorry for the delay. The KIP looks good to > me > > > and I have one follow up on AM3 regarding security for dlq topics. > > > > > > Should there be a validation with the user principal for the dlq topic > > > WRITE access from the topic being read i.e. if application is reading > from > > > T1 topic and dlq topic is dlq.T1 then should there be a check if > > > application has write permissions? The reason I am asking is that there > > > could be misconfigurations when another group uses > > > errors.deadletterqueue.topic.name to an already existing dlq topic, > which > > > can flood the other applications dlq topic. > > > > > > Regards, > > > Apoorv Mittal > > > > > > > > > On Tue, Jan 27, 2026 at 1:54 PM Andrew Schofield < > [email protected]> > > > wrote: > > > > > > > Hi PoAn, > > > > Thanks for your comment. > > > > > > > > PY00: I agree. I've made the changes to the KIP. > > > > > > > > Thanks, > > > > Andrew > > > > > > > > On 2026/01/15 10:16:18 PoAn Yang wrote: > > > > > Hi Andrew, > > > > > > > > > > Thanks for the KIP. I have a question about broker configuration. > > > > > > > > > > PY00: Would you consider mentioning the update mode for > > > > errors.deadletterqueue.topic.name.prefix > > > > > and errors.deadletterqueue.auto.create.topics.enable are > cluster-wide? > > > > > Clarifying that these values must be consistent across the cluster > (or > > > > updated dynamically as a cluster default) > > > > > would help preventing inconsistent values among brokers. > > > > > > > > > > Thanks, > > > > > PoAn > > > > > > > > > > > On Jan 8, 2026, at 6:18 PM, Andrew Schofield < > [email protected]> > > > > wrote: > > > > > > > > > > > > Hi Shekhar, > > > > > > Thanks for your comment. > > > > > > > > > > > > If the leader of the DLQ topic-partition changes as we are > trying to > > > > write to it, > > > > > > then the code will need to cope with this. > > > > > > > > > > > > If the leader of the share-partition changes, we do not need > special > > > > processing. > > > > > > If the transition to ARCHIVED is affected by a share-partition > > > > leadership change, > > > > > > the new leader will be responsible for the state transition. For > > > > example, if a consumer > > > > > > has rejected a record, a leadership change will cause the > rejection to > > > > fail, and the > > > > > > record will be delivered again. This new delivery attempt will be > > > > performed by the > > > > > > new leader, and if this delivery attempt results in a rejection, > the > > > > new leader will > > > > > > be responsible for initiating the DLQ write. > > > > > > > > > > > > Hope this makes sense, > > > > > > Andrew > > > > > > > > > > > > On 2026/01/03 15:02:31 Shekhar Prasad Rajak via dev wrote: > > > > > >> Hi, > > > > > >> If leader changes during DLQ write, or a share partition leader > > > > changes, the partition is marked FENCED and in-memory cache state is > lost, > > > > I think we need to add those cases as well. > > > > > >> Ref > > > > > https://github.com/apache/kafka/blob/trunk/core/src/main/java/kafka/server/share/SharePartitionManager.java#L857 > > > > > >> > > > > > >> > > > > > >> > > > > > >> Regards,Shekhar > > > > > >> > > > > > >> > > > > > >> > > > > > >> On Monday 29 December 2025 at 11:53:20 pm GMT+5:30, Andrew > > > > Schofield <[email protected]> wrote: > > > > > >> > > > > > >> Hi Abhinav, > > > > > >> Thanks for your comments. > > > > > >> > > > > > >> AD01: Even if we were to allow the client to write to the DLQ > topic, > > > > > >> it would not be sufficient for situations in which the problem > is one > > > > > >> that the client cannot handle. So, my view is that it's > preferable to > > > > > >> use the same mechanism for all DLQ topic writes, regardless of > > > > > >> whether the consumer initiated the process by rejecting a > > > > > >> record or not. > > > > > >> > > > > > >> AD02: I have added a metric for counting failed DLQ topic > produce > > > > > >> requests per group. The KIP does say that the broker logs an > > > > > >> error when it fails to produce to the DLQ topic. > > > > > >> > > > > > >> Thanks, > > > > > >> Andrew > > > > > >> > > > > > >> On 2025/12/16 10:38:39 Abhinav Dixit via dev wrote: > > > > > >>> Hi Andrew, > > > > > >>> Thanks for this KIP. I have a couple of questions - > > > > > >>> > > > > > >>> AD01: From an implementation perspective, why can't we > create/write > > > > records > > > > > >>> to the DLQ topic from the client? Why do we want to do it from > the > > > > broker? > > > > > >>> As far as I understand, archiving the record on the share > partition > > > > and > > > > > >>> writing records to DLQ are independent? As you've mentioned in > the > > > > KIP, "It > > > > > >>> is possible in rare situations that more than one DLQ record > could be > > > > > >>> written for a particular undeliverable record", won't we > minimize > > > > these > > > > > >>> scenarios (by eliminating the dependency on persister write > state > > > > result) > > > > > >>> by writing records to the DLQ from the client? > > > > > >>> > > > > > >>> AD02: I agree with AM01 that we should emit a metric which can > > > > report the > > > > > >>> count of failures of writing records to DLQ topic which an > > > > application > > > > > >>> developer can monitor. If we are logging an error, maybe we > should > > > > log the > > > > > >>> count of such failures periodically? > > > > > >>> > > > > > >>> Regards, > > > > > >>> Abhinav Dixit > > > > > >>> > > > > > >>> On Fri, Dec 12, 2025 at 3:08 AM Apoorv Mittal < > > > > [email protected]> > > > > > >>> wrote: > > > > > >>> > > > > > >>>> Hi Andrew, > > > > > >>>> Thanks for the much needed enhancement for SHare Groups. Some > > > > questions: > > > > > >>>> > > > > > >>>> AM1: The KIP states that in case of some failure "the broker > will > > > > log an > > > > > >>>> error", how an application developer will utilize this > information > > > > and know > > > > > >>>> about any such occurrences? Should we emit a metric which can > > > > report the > > > > > >>>> count of such failures which an application developer can > monitor? > > > > > >>>> > > > > > >>>> AM2: Today records can go to Archived state either when > exceeded the > > > > > >>>> delivery limit or explicitly rejected by the client. I am > expecting > > > > the > > > > > >>>> records will be written to dlq topic only in the former case > i.e. > > > > when > > > > > >>>> exceeded the delivery limit, that's what KIP explains. If > yes, then > > > > can't > > > > > >>>> there be a failure handling in the client which on > serialization or > > > > other > > > > > >>>> issues want to reject the message explicitly to be placed on > dlq? > > > > Should we > > > > > >>>> have a config which governs this behaviour i.e. if enabled > then any > > > > > >>>> explicitly rejected record from client will also go to dlq? > > > > > >>>> > > > > > >>>> AM3: I read your response on the thread related to the tricky > part > > > > of ACL > > > > > >>>> for DLQ topics and I have a question in the similar area. The > KIP > > > > defines a > > > > > >>>> config "errors.deadletterqueue.auto.create.topics.enable" > which if > > > > enabled > > > > > >>>> then broker can create the topic automatically using given > other > > > > dlq topic > > > > > >>>> params. If a new dlq topic is created then what basic > permissions > > > > should be > > > > > >>>> applied so the application developer can access? Should we > provide > > > > > >>>> capability to create dlq topics automatically or should > restrict > > > > that and > > > > > >>>> enforce it to be created by the application owner? By latter > we > > > > know the > > > > > >>>> application owner has access to the dlq topic already. > > > > > >>>> > > > > > >>>> AM4: For the "errors.deadletterqueue.topic.name.prefix", I am > > > > expecting > > > > > >>>> that this applies well for auto created dlq topics. But how > do we > > > > enforce > > > > > >>>> the prefix behaviour when the application developer provides > the > > > > dlq topic > > > > > >>>> name in group configuration? Will there be a check while > setting > > > > the group > > > > > >>>> configuration "errors.deadletterqueue.topic.name" as per > broker > > > > expected > > > > > >>>> prefix? > > > > > >>>> > > > > > >>>> Regards, > > > > > >>>> Apoorv Mittal > > > > > >>>> > > > > > >>>> > > > > > >>>> On Wed, Dec 10, 2025 at 5:59 PM Federico Valeri < > > > > [email protected]> > > > > > >>>> wrote: > > > > > >>>> > > > > > >>>>> Hi Andrew, a few comments/questions from me: > > > > > >>>>> > > > > > >>>>> FV00: The KIP says "copying of the original record data into > the > > > > DLQ > > > > > >>>>> is controlled by two configurations", but I only see the > client > > > > side > > > > > >>>>> configuration in the latest revision. > > > > > >>>>> > > > > > >>>>> FV01: The KIP says: "When an undeliverable record > transitions to > > > > the > > > > > >>>>> Archived state for such a group, a record is written onto > the DLQ > > > > > >>>>> topic". Later on it mentions a new "Archiving" state. Can you > > > > clarify > > > > > >>>>> the state transition when sending a record to a DLQ? > > > > > >>>>> > > > > > >>>>> FV02: Is the new state required to ensure that the DLQ > record is > > > > > >>>>> eventually written in case of the Share Coordinator failover? > > > > > >>>>> > > > > > >>>>> Thanks, > > > > > >>>>> Fede > > > > > >>>>> > > > > > >>>>> > > > > > >>>>> On Tue, Dec 2, 2025 at 7:19 PM Andrew Schofield < > > > > [email protected]> > > > > > >>>>> wrote: > > > > > >>>>>> > > > > > >>>>>> Hi, > > > > > >>>>>> I'd like to bump this discussion thread for adding DLQs to > share > > > > > >>>> groups. > > > > > >>>>>> > > > > > >>>>>> > > > > > >>>>> > > > > > >>>> > > > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-1191%3A+Dead-letter+queues+for+share+groups > > > > > >>>>>> > > > > > >>>>>> Thanks, > > > > > >>>>>> Andrew > > > > > >>>>>> > > > > > >>>>>> On 2025/10/16 19:02:48 Andrew Schofield wrote: > > > > > >>>>>>> Hi Chia-Ping, > > > > > >>>>>>> Apologies for not responding to your comments. I was > having email > > > > > >>>>> problems > > > > > >>>>>>> and I’ve only just noticed the unanswered comments. Also, > this is > > > > > >>>> not a > > > > > >>>>>>> direct reply. > > > > > >>>>>>> > > > > > >>>>>>>>> chia00: How can we specify the number of partitions and > the > > > > > >>>>> replication factor > > > > > >>>>>>> when `errors.deadletterqueue.auto.create.topics.enable` > is set > > > > to > > > > > >>>>> true? > > > > > >>>>>>> > > > > > >>>>>>> Personally, I prefer to make people create their DLQ topics > > > > manually, > > > > > >>>>> but I take the > > > > > >>>>>>> point. In order to give full flexibility, the list of > configs you > > > > > >>>> need > > > > > >>>>> is quite long including > > > > > >>>>>>> min.isr and compression. For consistency with Kafka > Connect sink > > > > > >>>>> connectors, I > > > > > >>>>>>> could add > `errors.deadletterqueue.topic.replication.factor` but > > > > > >>>> that's > > > > > >>>>> the only > > > > > >>>>>>> additional config provided by Kafka Connect. Is that > worthwhile? > > > > I > > > > > >>>>> suggest not. > > > > > >>>>>>> > > > > > >>>>>>> The DLQ topic config in this KIP is broker-level config, > while > > > > it's > > > > > >>>>> connector-level > > > > > >>>>>>> config for Kafka Connect. So, my preference is to just > have one > > > > > >>>>> broker-level config > > > > > >>>>>>> for auto-creation on/off, and auto-create with the > cluster's > > > > topic > > > > > >>>>> defaults. If anything > > > > > >>>>>>> more specific is required, the administrator can create > the DLQ > > > > topic > > > > > >>>>> themselves with > > > > > >>>>>>> their preferences. Let me know what you think. > > > > > >>>>>>> > > > > > >>>>>>>>> chia01: Should the error stack trace be included in the > message > > > > > >>>>> headers, > > > > > >>>>>>> similar to what's done in KIP-298? > > > > > >>>>>>> > > > > > >>>>>>> In KIP-298, the code deciding to write a message to the > DLQ is > > > > > >>>> running > > > > > >>>>> in the > > > > > >>>>>>> Kafka Connect task and an exception is readily available. > In this > > > > > >>>> KIP, > > > > > >>>>> the code writing > > > > > >>>>>>> to the DLQ is running in the broker and it doesn't have any > > > > detail > > > > > >>>>> about why the > > > > > >>>>>>> record is being DLQed. I think that actually the > > > > > >>>>> __dlq.errors.exception.* headers > > > > > >>>>>>> are not feasible without allowing the application to > provide > > > > > >>>>> additional error context. > > > > > >>>>>>> That might be helpful one day, but that's extending this > KIP more > > > > > >>>> than > > > > > >>>>> I intend. > > > > > >>>>>>> I have removed these headers from the KIP. > > > > > >>>>>>> > > > > > >>>>>>>>> chia02: Why does > `errors.deadletterqueue.copy.record.enable` > > > > have > > > > > >>>>> different > > > > > >>>>>>> default values at the broker level and group level? > > > > > >>>>>>> > > > > > >>>>>>> I want the group administrator to be able to choose > whether to > > > > copy > > > > > >>>>> the payloads. > > > > > >>>>>>> I was also thinking that it would be a good idea if the > cluster > > > > > >>>>> administrator could > > > > > >>>>>>> prevent this across the cluster, but I've changed my mind > and > > > > I've > > > > > >>>>> removed it. > > > > > >>>>>>> > > > > > >>>>>>> Maybe a better idea would simply to have a broker config > > > > > >>>>>>> `group.share.errors.deadletterqueue.enable` to turn DLQ > on/off. > > > > The > > > > > >>>>> other > > > > > >>>>>>> broker configs in this KIP do not start `group.share.` > because > > > > > >>>> they're > > > > > >>>>> intended > > > > > >>>>>>> for other DLQ uses by the broker in future. > > > > > >>>>>>> > > > > > >>>>>>> Note that although share.version=2 is required to enable > DLQ, > > > > this > > > > > >>>>> isn't a suitable > > > > > >>>>>>> long-term switch because we might have share.version > 2 > due to > > > > > >>>>> another future > > > > > >>>>>>> enhancement. > > > > > >>>>>>> > > > > > >>>>>>>>> chia03: Does the broker log an error for every message > if the > > > > DLQ > > > > > >>>>> topic fails to be created? > > > > > >>>>>>> > > > > > >>>>>>> No, that seems excessive and likely to flood the logs. I > would > > > > > >>>>> implement something like > > > > > >>>>>>> no more than one log per minute, per share-partition. That > would > > > > be > > > > > >>>>> annoying enough to > > > > > >>>>>>> fix without being catastrophically verbose. > > > > > >>>>>>> > > > > > >>>>>>> Of course, if the group config ` > > > > errors.deadletterqueue.topic.name` > > > > > >>>>> has a value which > > > > > >>>>>>> does not satisfy the broker config > > > > > >>>>> `errors.deadletterqueue.topic.name.prefix`, it will > > > > > >>>>>>> be considered a config error and the DLQ will not be used. > > > > > >>>>>>> > > > > > >>>>>>>>> chia04: Have you consider adding metrics for the DLQ? > > > > > >>>>>>> > > > > > >>>>>>> Yes, that is a good idea. I've added some metrics to the > KIP. > > > > Please > > > > > >>>>> take a look. > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> Thanks, > > > > > >>>>>>> Andrew > > > > > >>>>>>> > > > > > >>>>>>>> On 4 Aug 2025, at 11:30, Andrew Schofield < > > > > > >>>>> [email protected]> wrote: > > > > > >>>>>>>> > > > > > >>>>>>>> Hi, > > > > > >>>>>>>> Thanks for your comments on the KIP and sorry for the > delay in > > > > > >>>>> responding. > > > > > >>>>>>>> > > > > > >>>>>>>> D01: Authorisation is the area of this KIP that I think > is most > > > > > >>>>> tricky. The reason that I didn't implement specific > > > > > >>>>>>>> ACLs for DLQs because I was not convinced they would > help. So, > > > > if > > > > > >>>>> you have a specific idea in mind, please > > > > > >>>>>>>> let me know. This is the area that I'm least comfortable > with in > > > > > >>>> the > > > > > >>>>> KIP. > > > > > >>>>>>>> > > > > > >>>>>>>> I suppose maybe to set the DLQ name for a group, you > could need > > > > a > > > > > >>>>> higher level of authorisation > > > > > >>>>>>>> than just ALTER_CONFIGS on the GROUP. But what I settled > with in > > > > > >>>> the > > > > > >>>>> KIP was that DLQ topics > > > > > >>>>>>>> all start with the same prefix, defaulting to "dlq.", and > that > > > > the > > > > > >>>>> topics do not automatically create. > > > > > >>>>>>>> > > > > > >>>>>>>> D02: I can see that. I've added a config which I've called > > > > > >>>>> errors.deadletterqueue.auto.create.topics.enable > > > > > >>>>>>>> just to have a consistent prefix on all of the config > names. > > > > Let me > > > > > >>>>> know what you think. > > > > > >>>>>>>> > > > > > >>>>>>>> D03: I've added some text about failure scenarios when > > > > attempting > > > > > >>>> to > > > > > >>>>> write records to the DLQ. > > > > > >>>>>>>> > > > > > >>>>>>>> Thanks, > > > > > >>>>>>>> Andrew > > > > > >>>>>>>> ________________________________________ > > > > > >>>>>>>> From: isding_l <[email protected]> > > > > > >>>>>>>> Sent: 16 July 2025 04:18 > > > > > >>>>>>>> To: dev <[email protected]> > > > > > >>>>>>>> Subject: Re: [DISCUSS]: KIP-1191: Dead-letter queues for > share > > > > > >>>> groups > > > > > >>>>>>>> > > > > > >>>>>>>> Hi Andrew, > > > > > >>>>>>>> Thanks for the nice KIP, This KIP design for introducing > > > > > >>>> dead-letter > > > > > >>>>> queues (DLQs) for Share Groups is generally clear and > reasonable, > > > > > >>>>> addressing the key pain points of handling "poison message". > > > > > >>>>>>>> > > > > > >>>>>>>> > > > > > >>>>>>>> D01: Should we consider implementing independent ACL > > > > configurations > > > > > >>>>> for DLQs? This would enable separate management of DLQ topic > > > > read/write > > > > > >>>>> permissions from source topics, preventing privilege > escalation > > > > attacks > > > > > >>>> via > > > > > >>>>> "poison message" + DLQ mechanisms. > > > > > >>>>>>>> > > > > > >>>>>>>> > > > > > >>>>>>>> D02: While disabling automatic DLQ topic creation is > justifiable > > > > > >>>> for > > > > > >>>>> security, it creates operational overhead in automated > > > > deployments. Can > > > > > >>>> we > > > > > >>>>> introduce a configuration parameter > auto.create.dlq.topics.enable > > > > to > > > > > >>>> govern > > > > > >>>>> this behavior? > > > > > >>>>>>>> > > > > > >>>>>>>> > > > > > >>>>>>>> D03: How should we handle failure scenarios when brokers > > > > attempt to > > > > > >>>>> write records to the DLQ? > > > > > >>>>>>>> ---- Replied Message ---- > > > > > >>>>>>>> | From | Andrew Schofield< > [email protected]> | > > > > > >>>>>>>> | Date | 07/08/2025 17:54 | > > > > > >>>>>>>> | To | [email protected]<[email protected]> | > > > > > >>>>>>>> | Subject | [DISCUSS]: KIP-1191: Dead-letter queues for > share > > > > > >>>> groups > > > > > >>>>> | > > > > > >>>>>>>> Hi, > > > > > >>>>>>>> I'd like to start discussion on KIP-1191 which adds > dead-letter > > > > > >>>>> queue support for share groups. > > > > > >>>>>>>> Records which cannot be processed by consumers in a share > group > > > > can > > > > > >>>>> be automatically copied > > > > > >>>>>>>> onto another topic for a closer look. > > > > > >>>>>>>> > > > > > >>>>>>>> KIP: > > > > > >>>>> > > > > > >>>> > > > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-1191%3A+Dead-letter+queues+for+share+groups > > > > > >>>>>>>> > > > > > >>>>>>>> Thanks, > > > > > >>>>>>>> Andrew > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>> > > > > > >>>> > > > > > >>> > > > > > >> > > > > > > > > > > > > > > > > > > > >
