eployments. 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 |
| Date | 07/08/2025 17:54 |
| To |
Hi Corki,
Thanks for the KIP.
You need to assign a number to your KIP. I think it should be 1199. You can
find instructions here (1198 already exists):
https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals
If you need help, let me know.
Thanks,
Andrew
_
[
https://issues.apache.org/jira/browse/KAFKA-16717?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Andrew Schofield resolved KAFKA-16717.
--
Fix Version/s: 4.2.0
Resolution: Fixed
>
[
https://issues.apache.org/jira/browse/KAFKA-19471?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Andrew Schofield resolved KAFKA-19471.
--
Resolution: Fixed
> Enable acknowledgement for a record which could not
Hi,
I downloaded and tested 4.1.0 RC1.
I successfully checked the enablement and administration of the share
group and streams groups features.
I also downloaded the documentation from the pending kafka-site PR.
I observe that it was necessary to manually change the #include
directives from 40 to
Hi Viktor,
Thanks for the KIP. Personally, I like large KIPs to be published while there
are still details to
be worked out because it gives an opportunity to do the discussion in the
community.
I like the idea of being able to subdivide and virtualise clusters, and I agree
that it would
be goo
rote:
> Thanks Andrew for the KIP. The inconsistency always drives me crazy.
> +1 (binding)
>
> Regards,
> Omnia
> > On 21 Jul 2025, at 09:48, Andrew Schofield <
> andrew_schofield_j...@outlook.com> wrote:
> >
> > Hi,
> > I would like to start a vot
Hi,
I would like to start a vote for KIP-1147 which improves the consistency of
command-line arguments of the various Kafka CLI tools.
https://cwiki.apache.org/confluence/display/KAFKA/KIP-1147%3A+Improve+consistency+of+command-line+arguments
Thanks,
Andrew
[
https://issues.apache.org/jira/browse/KAFKA-19500?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Andrew Schofield resolved KAFKA-19500.
--
Fix Version/s: 4.2.0
Resolution: Fixed
> kafka-consumer-groups.sh should f
[
https://issues.apache.org/jira/browse/KAFKA-19461?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Andrew Schofield resolved KAFKA-19461.
--
Fix Version/s: 4.2.0
Resolution: Fixed
> Add share group admin integrat
Hi,
Thanks for the KIP.
+1 (binding)
Andrew
From: Federico Valeri
Sent: 18 July 2025 11:03
To: dev@kafka.apache.org
Subject: [VOTE] KIP-1192: Add include argument to ConsumerPerformance tool
Hello, I would like to call a vote for KIP-1192:
https://cwi
Hi Fede (and Kirk),
Thanks for the KIP. A comment on Kirk's comment :)
KT01: This is certainly an interesting question. I think we need a small KIP
for kafka-console-consumer.sh
because it only supports Consumer.subscribe(Pattern), and not the newer KIP-848
Consumer.subscribe(SubscriptionPattern)
Congratulations Lucas.
From: Apoorv Mittal
Sent: 15 July 2025 10:48
To: dev@kafka.apache.org
Subject: Re: [ANNOUNCE] New Kafka PMC member: Lucas Brutschy
Congratulations Lucas!
Regards,
Apoorv Mittal
On Tue, Jul 15, 2025 at 9:53 AM Bruno Cadonna wro
operty), which takes
precedence?
I'm not ecstatic about the existing naming of "--command-config" because it's
ambiguous, but that's a battle for another day...
Thanks,
Kirk
On Wed, Jul 9, 2025, at 6:53 AM, Andrew Schofield wrote:
> Hi,
> I'd like to start
ence?
>
> I'm not ecstatic about the existing naming of "--command-config" because
> it's ambiguous, but that's a battle for another day...
>
> Thanks,
> Kirk
>
> On Wed, Jul 9, 2025, at 6:53 AM, Andrew Schofield wrote:
> > Hi,
> > I'd l
Great news. Well done PoAn.
From: Lianet M.
Sent: 13 July 2025 20:48
To: dev@kafka.apache.org
Subject: Re: [ANNOUNCE] New Kafka Committer: PoAn Yang
Glad to hear! Congrats PoAn!!!
On Sun, Jul 13, 2025, 8:53 p.m. Gaurav Narula wrote:
> Congratulations
eType type);
Thanks,
Jun
On Fri, Jul 11, 2025 at 2:30 AM Andrew Schofield <
andrew_schofield_j...@outlook.com> wrote:
> Hi,
> As we've written the code for KIP-932, we've made some small changes to
> the KIP
> to correct mistakes, align with other accepted KIPs and
t 2024 19:07
To: dev@kafka.apache.org
Subject: Re: [VOTE] KIP-932: Queues for Kafka
Hi, Andrew,
Thanks for the clarification. Sounds good.
Jun
On Thu, Aug 15, 2024 at 2:53 AM Andrew Schofield
wrote:
> Hi Jun,
> Thanks for the questions.
>
> 1. The PartitionMetadata is only
Thanks for the KIP.
+1 (binding)
Andrew
From: Junwang Guo
Sent: 09 July 2025 14:04
To: dev@kafka.apache.org
Subject: [VOTE] KIP-1172: Improve EndToEndLatency tool
Hi all,
I would like to start a vote on KIP-1172: Improve EndToEndLatency tool.
KIP: ht
rning
will also be added to guide users to the new syntax.
AS2: Thanks for pointing that out. I'll update the parameter names as
suggested.
Appreciate your comments!
Best,
Nick
On Wed, Jul 9, 2025 at 9:28 PM Andrew Schofield <
andrew_schofield_j...@outlook.com> wrote:
> Hi Ni
Andrew Schofield created KAFKA-19487:
Summary: Improve consistency of command-line arguments
Key: KAFKA-19487
URL: https://issues.apache.org/jira/browse/KAFKA-19487
Project: Kafka
Issue
Hi,
I'd like to start discussion of KIP-1147. This KIP aligns the names of the
command-line
arguments across all of the Apache Kafka command-line tools.
KIP:
https://cwiki.apache.org/confluence/display/KAFKA/KIP-1147%3A+Improve+consistency+of+command-line+arguments
Thanks,
Andrew
Hi Nick,
Thanks for the KIP.
Sorry for the late discussion. I was reading the KIP, nudged by the voting
thread and I
have a couple of comments.
AS1: The old syntax should eventually not be supported. Please could the KIP say
something like "The previous syntax using positional arguments is depre
[
https://issues.apache.org/jira/browse/KAFKA-19457?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Andrew Schofield resolved KAFKA-19457.
--
Fix Version/s: 4.2.0
Resolution: Fixed
> Revisit the threshold to re
[
https://issues.apache.org/jira/browse/KAFKA-19450?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Andrew Schofield resolved KAFKA-19450.
--
Resolution: Fixed
> ShareConsumerPerformance does not handle exceptions f
[
https://issues.apache.org/jira/browse/KAFKA-19463?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Andrew Schofield resolved KAFKA-19463.
--
Fix Version/s: 4.2.0
Resolution: Fixed
> nextFetchOffset does not take ongo
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%3
Andrew Schofield created KAFKA-19471:
Summary: Enable acknowledgement for a record which could not be
deserialized
Key: KAFKA-19471
URL: https://issues.apache.org/jira/browse/KAFKA-19471
Project
[
https://issues.apache.org/jira/browse/KAFKA-19468?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Andrew Schofield resolved KAFKA-19468.
--
Resolution: Fixed
> Share group epoch increments on every heartbeat if subscri
[
https://issues.apache.org/jira/browse/KAFKA-19363?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Andrew Schofield resolved KAFKA-19363.
--
Resolution: Fixed
> Enhance share group assignor for heterogeneous subscription t
[
https://issues.apache.org/jira/browse/KAFKA-19362?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Andrew Schofield resolved KAFKA-19362.
--
Resolution: Fixed
> Enhance share group assignor for homogeneous subscription t
Andrew Schofield created KAFKA-19468:
Summary: Share group epoch increments on every heartbeat if
subscribed topics removed
Key: KAFKA-19468
URL: https://issues.apache.org/jira/browse/KAFKA-19468
[
https://issues.apache.org/jira/browse/KAFKA-19440?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Andrew Schofield resolved KAFKA-19440.
--
Fix Version/s: 4.2.0
Resolution: Fixed
> Admin.alterShareGroupOffsets does
Andrew Schofield
wrote:
>
> Hi Mickael,
> We have found an NPE in the group coordinator while stress testing which
> should probably be fixed.
> There is a chance that a topic being looked up is not present in the metadata
> image
> in GroupCoordinatorService.onPartitionsDeleted
Hi Mickael,
We have found an NPE in the group coordinator while stress testing which should
probably be fixed.
There is a chance that a topic being looked up is not present in the metadata
image
in GroupCoordinatorService.onPartitionsDeleted. This is KAFKA-19454.
Here's the trunk PR https://gith
Andrew Schofield created KAFKA-19461:
Summary: Add share group admin integration tests to
PlaintextAdminIntegrationTest
Key: KAFKA-19461
URL: https://issues.apache.org/jira/browse/KAFKA-19461
Andrew Schofield created KAFKA-19440:
Summary: Admin.alterShareGroupOffsets doesn't handle exceptions
correctly
Key: KAFKA-19440
URL: https://issues.apache.org/jira/browse/KAFKA-19440
Pr
Andrew Schofield created KAFKA-19437:
Summary: Review incidence of null metadata image in share
coordinator
Key: KAFKA-19437
URL: https://issues.apache.org/jira/browse/KAFKA-19437
Project: Kafka
Congrats, Omnia. Very good to hear.
From: Lianet M.
Sent: 25 June 2025 17:57
To: dev@kafka.apache.org
Subject: Re: [ANNOUNCE] New Kafka Committer: Omnia Ibrahim
Congrats Omnia!!!
On Wed, Jun 25, 2025, 12:48 p.m. Kamal Chandraprakash <
kamal.chandraprak.
Hi,
Thanks for the KIP. Since we are making more use of features these days, this
is timely.
+1 (binding)
Thanks,
Andrew
From: PoAn Yang
Sent: 25 June 2025 07:18
To: dev@kafka.apache.org
Subject: [VOTE] KIP-1160: Enable returning supported features fro
[
https://issues.apache.org/jira/browse/KAFKA-19416?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Andrew Schofield resolved KAFKA-19416.
--
Fix Version/s: 4.2.0
Resolution: Fixed
> Reduce running time
Andrew Schofield created KAFKA-19416:
Summary: Reduce running time of ShareConsumerTest
Key: KAFKA-19416
URL: https://issues.apache.org/jira/browse/KAFKA-19416
Project: Kafka
Issue Type
Andrew Schofield created KAFKA-19410:
Summary: Validate alignment of subscribed topics and assigned
partitions in share consumer
Key: KAFKA-19410
URL: https://issues.apache.org/jira/browse/KAFKA-19410
[
https://issues.apache.org/jira/browse/KAFKA-19369?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Andrew Schofield resolved KAFKA-19369.
--
Fix Version/s: 4.1.0
Resolution: Fixed
> Add group.share.assignors con
[
https://issues.apache.org/jira/browse/KAFKA-19290?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Andrew Schofield resolved KAFKA-19290.
--
Fix Version/s: 4.1.0
Resolution: Fixed
> Exploit mapKey optimisation
[
https://issues.apache.org/jira/browse/KAFKA-19370?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Andrew Schofield resolved KAFKA-19370.
--
Fix Version/s: 4.1.0
Resolution: Fixed
> Create JMH benchmark for assess
+1 (binding)
Thanks,
Andrew
From: TengYao Chi
Sent: 05 June 2025 17:05
To: dev@kafka.apache.org
Subject: Re: [VOTE] KIP-1175: Fix the typo
`PARTITIONER_ADPATIVE_PARTITIONING_ENABLE` in ProducerConfig
+1 (binding)
Best,
TYC
Chia-Ping Tsai 於 2025年6月5日 週
[
https://issues.apache.org/jira/browse/KAFKA-19328?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Andrew Schofield resolved KAFKA-19328.
--
Fix Version/s: 4.1.0
Resolution: Resolved
Test removed
Andrew Schofield created KAFKA-19370:
Summary: Create JMH benchmark for assessing performance of share
group assignor
Key: KAFKA-19370
URL: https://issues.apache.org/jira/browse/KAFKA-19370
Andrew Schofield created KAFKA-19369:
Summary: Add group.share.assignors config
Key: KAFKA-19369
URL: https://issues.apache.org/jira/browse/KAFKA-19369
Project: Kafka
Issue Type: Sub
Andrew Schofield created KAFKA-19364:
Summary: Production-ready documentation
Key: KAFKA-19364
URL: https://issues.apache.org/jira/browse/KAFKA-19364
Project: Kafka
Issue Type: Sub-task
Andrew Schofield created KAFKA-19363:
Summary: Enhance share group assignor for heterogeneous
subscription type
Key: KAFKA-19363
URL: https://issues.apache.org/jira/browse/KAFKA-19363
Project
Andrew Schofield created KAFKA-19362:
Summary: Enhance share group assignor for homogeneous subscription
type
Key: KAFKA-19362
URL: https://issues.apache.org/jira/browse/KAFKA-19362
Project
[
https://issues.apache.org/jira/browse/KAFKA-19353?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Andrew Schofield resolved KAFKA-19353.
--
Fix Version/s: 4.1.0
Resolution: Fixed
> Preview documentat
[
https://issues.apache.org/jira/browse/KAFKA-19338?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Andrew Schofield resolved KAFKA-19338.
--
Fix Version/s: 4.1.0
Resolution: Fixed
> Share coordinator read state
[
https://issues.apache.org/jira/browse/KAFKA-19358?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Andrew Schofield resolved KAFKA-19358.
--
Fix Version/s: 4.1.0
Resolution: Fixed
> Do not update share group off
[
https://issues.apache.org/jira/browse/KAFKA-19320?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Andrew Schofield resolved KAFKA-19320.
--
Fix Version/s: 4.1.0
Resolution: Fixed
> Add share_consume_bench_test
[
https://issues.apache.org/jira/browse/KAFKA-16894?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Andrew Schofield resolved KAFKA-16894.
--
Resolution: Fixed
> Define switch to enable share groups for prev
[
https://issues.apache.org/jira/browse/KAFKA-19321?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Andrew Schofield resolved KAFKA-19321.
--
Fix Version/s: 4.1.0
Resolution: Fixed
> Add share_consumer_performance.py
Andrew Schofield created KAFKA-19353:
Summary: Preview documentation
Key: KAFKA-19353
URL: https://issues.apache.org/jira/browse/KAFKA-19353
Project: Kafka
Issue Type: Sub-task
[
https://issues.apache.org/jira/browse/KAFKA-19337?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Andrew Schofield resolved KAFKA-19337.
--
Fix Version/s: 4.1.0
Resolution: Fixed
> Bug
[
https://issues.apache.org/jira/browse/KAFKA-19316?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Andrew Schofield resolved KAFKA-19316.
--
Fix Version/s: 4.1.0
Resolution: Fixed
> Add share_group_command_test
[
https://issues.apache.org/jira/browse/KAFKA-19344?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Andrew Schofield resolved KAFKA-19344.
--
Fix Version/s: 4.1.0
Resolution: Fixed
> Move assignable share partit
[
https://issues.apache.org/jira/browse/KAFKA-19342?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Andrew Schofield resolved KAFKA-19342.
--
Fix Version/s: 4.1.0
Resolution: Fixed
> Add AuthorizerIntegrationTest
Andrew Schofield created KAFKA-19342:
Summary: Add AuthorizerIntegrationTest for alter share group
offsets
Key: KAFKA-19342
URL: https://issues.apache.org/jira/browse/KAFKA-19342
Project: Kafka
[
https://issues.apache.org/jira/browse/KAFKA-19268?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Andrew Schofield resolved KAFKA-19268.
--
Fix Version/s: 4.1.0
Resolution: Fixed
> Missing mocks
Andrew Schofield created KAFKA-19290:
Summary: Exploit mapKey optimisation in protocol requests and
responses
Key: KAFKA-19290
URL: https://issues.apache.org/jira/browse/KAFKA-19290
Project
Andrew Schofield created KAFKA-19286:
Summary: RemoteIndexCacheTest.testConcurrentRemoveReadForCache1 is
flaky
Key: KAFKA-19286
URL: https://issues.apache.org/jira/browse/KAFKA-19286
Project
Hi Peter and David,
Thanks for the KIP.
It contains some interesting ideas but it's very nebulous at this point. I
think the suggestion of a proxy
layer in front of Kafka is a good way to start probing at this. Are the
existing Kafka metrics helpful
for QoS? What changes might we make to the Kaf
on.fetch.bytes, or does it handle the larger
response gracefully?
Thanks,
Kamal
On Thu, May 8, 2025 at 7:19 PM Andrew Schofield <
andrew_schofield_j...@outlook.com> wrote:
> Hi Kamal,
> Thanks for the KIP.
>
> While it makes a lot of sense to me to be able to control the fetchin
[
https://issues.apache.org/jira/browse/KAFKA-19201?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Andrew Schofield resolved KAFKA-19201.
--
Fix Version/s: 4.1.0
Resolution: Fixed
> Handle deletion of a topic which
[
https://issues.apache.org/jira/browse/KAFKA-19245?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Andrew Schofield resolved KAFKA-19245.
--
Resolution: Fixed
> Update defaults for preview of queues configurati
[
https://issues.apache.org/jira/browse/KAFKA-19051?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Andrew Schofield resolved KAFKA-19051.
--
Resolution: Won't Fix
Reviewing this issue, it is clear that the behaviour sugg
Hi Ming-Yen,
Thanks for the KIP. I'm glad to see that this is going to be cleaned up :)
AS1: There's no need to deprecate the DOC constant because it's private.
Thanks,
Andrew
From: 鍾明諺
Sent: 08 May 2025 09:21
To: dev@kafka.apache.org
Subject: [DISCUSS]
Hi Kamal,
Thanks for the KIP.
While it makes a lot of sense to me to be able to control the fetching from
remote
storage to make sure it's sympathetic to the characteristics of the storage
provider,
it seems to me that extending this concept all the way to the individual
consumers
is not a good
[
https://issues.apache.org/jira/browse/KAFKA-19218?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Andrew Schofield resolved KAFKA-19218.
--
Fix Version/s: 4.1.0
Resolution: Fixed
> Add missing leader epoch to r
[
https://issues.apache.org/jira/browse/KAFKA-19216?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Andrew Schofield resolved KAFKA-19216.
--
Fix Version/s: 4.1.0
Resolution: Fixed
> Eliminate flakiness
[
https://issues.apache.org/jira/browse/KAFKA-19015?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Andrew Schofield resolved KAFKA-19015.
--
Fix Version/s: 4.1.0
Resolution: Fixed
> Remove share session from cache
[
https://issues.apache.org/jira/browse/KAFKA-19169?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Andrew Schofield resolved KAFKA-19169.
--
Fix Version/s: 4.1.0
Resolution: Fixed
> Enhance AuthorizerIntegrationTest
[
https://issues.apache.org/jira/browse/KAFKA-16718?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Andrew Schofield resolved KAFKA-16718.
--
Fix Version/s: 4.1.0
Resolution: Fixed
>
Andrew Schofield created KAFKA-19218:
Summary: Add missing leader epoch to read share group state
summary response
Key: KAFKA-19218
URL: https://issues.apache.org/jira/browse/KAFKA-19218
Project
Andrew Schofield created KAFKA-19217:
Summary: ShareConsumerTest.testComplexShareConsumer is unreliable
Key: KAFKA-19217
URL: https://issues.apache.org/jira/browse/KAFKA-19217
Project: Kafka
Andrew Schofield created KAFKA-19216:
Summary: Eliminate flakiness in
kafka.server.share.SharePartitionTest
Key: KAFKA-19216
URL: https://issues.apache.org/jira/browse/KAFKA-19216
Project: Kafka
Thanks for the KIP, Ivan.
+1 (binding)
Thanks,
Andrew
From: Josep Prat
Sent: 29 April 2025 11:57
To: dev@kafka.apache.org
Subject: Re: [VOTE] KIP-1123: Rack-aware partitioning for Kafka Producer
Thanks for the KIP Ivan!
+1 binding
Best,
On Tue, Apr 22
nspired by KIP-1152, have you
considered adding support for pattern filters? It seems to me that the issue
encountered by KIP-1152 could also happen with group IDs.
Best,
Chia-Ping
On 2024/06/04 17:08:10 Andrew Schofield wrote:
> Hi,
> I would like to start a discussion thread on KIP-1043: A
Andrew Schofield created KAFKA-19204:
Summary: Timeouts in coordinator runtime operations can break
share group initialization and deletion
Key: KAFKA-19204
URL: https://issues.apache.org/jira/browse/KAFKA
[
https://issues.apache.org/jira/browse/KAFKA-19198?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Andrew Schofield resolved KAFKA-19198.
--
Fix Version/s: 4.1.0
Resolution: Fixed
> Deletion of topic assigned to a sh
Andrew Schofield created KAFKA-19201:
Summary: Handle deletion of a topic which has initialised
share-group state
Key: KAFKA-19201
URL: https://issues.apache.org/jira/browse/KAFKA-19201
Project
Andrew Schofield created KAFKA-19198:
Summary: Deletion of topic assigned to a share group can result in
NPE
Key: KAFKA-19198
URL: https://issues.apache.org/jira/browse/KAFKA-19198
Project: Kafka
Andrew Schofield created KAFKA-19189:
Summary: Handle deletion of __share_group_state topic in share
coordinator
Key: KAFKA-19189
URL: https://issues.apache.org/jira/browse/KAFKA-19189
Project
[
https://issues.apache.org/jira/browse/KAFKA-18629?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Andrew Schofield resolved KAFKA-18629.
--
Fix Version/s: 4.1.0
Resolution: Fixed
> DeleteShareGroups i
[
https://issues.apache.org/jira/browse/KAFKA-19156?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Andrew Schofield resolved KAFKA-19156.
--
Fix Version/s: 4.1.0
Resolution: Fixed
> Remove group.share.max.groups con
[
https://issues.apache.org/jira/browse/KAFKA-18488?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Andrew Schofield resolved KAFKA-18488.
--
Resolution: Fixed
https://issues.apache.org/jira/browse/KAFKA-18794 tracks the
[
https://issues.apache.org/jira/browse/KAFKA-19057?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Andrew Schofield resolved KAFKA-19057.
--
Fix Version/s: 4.1.0
Resolution: Fixed
> Stabilize KIP-932 Kafka protocol
Andrew Schofield created KAFKA-19169:
Summary: Enhance AuthorizerIntegrationTest for share group APIs
Key: KAFKA-19169
URL: https://issues.apache.org/jira/browse/KAFKA-19169
Project: Kafka
[
https://issues.apache.org/jira/browse/KAFKA-19024?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Andrew Schofield reopened KAFKA-19024:
--
Assignee: Andrew Schofield (was: Lan Ding)
Reopened to update the Resolution field
[
https://issues.apache.org/jira/browse/KAFKA-19024?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Andrew Schofield resolved KAFKA-19024.
--
Resolution: Won't Fix
The `group.share.max.groups` configuration was removed so
[
https://issues.apache.org/jira/browse/KAFKA-19158?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Andrew Schofield resolved KAFKA-19158.
--
Fix Version/s: 4.1.0
Resolution: Fixed
> Add a new error code for “sh
+1 (binding)
Thanks,
Andrew
From: Lucas Brutschy
Sent: 16 April 2025 08:17
To: dev@kafka.apache.org
Subject: Re: [VOTE] KIP-1101: Trigger rebalance on rack topology changes
+1 (binding)
On Tue, Dec 17, 2024 at 2:08 AM 郭骏旺 wrote:
>
> +1 (non-binding)
>
[
https://issues.apache.org/jira/browse/KAFKA-18902?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Andrew Schofield resolved KAFKA-18902.
--
Resolution: Fixed
> Implement ShareConsumer option to throw on poll if there
1 - 100 of 638 matches
Mail list logo