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 > >
