Re: [DISCUSS]: KIP-1191: Dead-letter queues for share groups

2025-08-04 Thread Andrew Schofield
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 |

Re: [DISCUSS] KIP-XXXX: Add max record count limit to FetchRequest

2025-07-29 Thread Andrew Schofield
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 _

[jira] [Resolved] (KAFKA-16717) Add AdminClient.alterShareGroupOffsets

2025-07-29 Thread Andrew Schofield (Jira)
[ 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 >

[jira] [Resolved] (KAFKA-19471) Enable acknowledgement for a record which could not be deserialized

2025-07-27 Thread Andrew Schofield (Jira)
[ 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

Re: [VOTE] 4.1.0 RC1

2025-07-26 Thread Andrew Schofield
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

Re: 回复: [DISCUSS] KIP-1134: Virtual Clusters in Kafka

2025-07-24 Thread Andrew Schofield
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

Re: [VOTE] KIP-1147: Improve consistency of command-line arguments

2025-07-22 Thread Andrew Schofield
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

[VOTE] KIP-1147: Improve consistency of command-line arguments

2025-07-21 Thread Andrew Schofield
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

[jira] [Resolved] (KAFKA-19500) kafka-consumer-groups.sh should fail quickly if the partition leader is unavailable

2025-07-21 Thread Andrew Schofield (Jira)
[ 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

[jira] [Resolved] (KAFKA-19461) Add share group admin integration tests to PlaintextAdminIntegrationTest

2025-07-21 Thread Andrew Schofield (Jira)
[ 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

Re: [VOTE] KIP-1192: Add include argument to ConsumerPerformance tool

2025-07-19 Thread Andrew Schofield
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

Re: [DISCUSS] KIP-1192: Add include argument to ConsumerPerformance tool

2025-07-15 Thread Andrew Schofield
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)

Re: [ANNOUNCE] New Kafka PMC member: Lucas Brutschy

2025-07-15 Thread Andrew Schofield
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

Re: [DISCUSS] KIP-1147: Improve consistency of command-line arguments

2025-07-14 Thread Andrew Schofield
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

Re: [DISCUSS] KIP-1147: Improve consistency of command-line arguments

2025-07-14 Thread Andrew Schofield
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

Re: [ANNOUNCE] New Kafka Committer: PoAn Yang

2025-07-13 Thread Andrew Schofield
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

Re: [VOTE] KIP-932: Queues for Kafka

2025-07-12 Thread Andrew Schofield
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

Re: [VOTE] KIP-932: Queues for Kafka

2025-07-11 Thread Andrew Schofield
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

Re: [VOTE] KIP-1172: Improve EndToEndLatency tool

2025-07-10 Thread Andrew Schofield
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

Re: [DISCUSS] KIP-1172: Improve EndToEndLatency tool

2025-07-09 Thread Andrew Schofield
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

[jira] [Created] (KAFKA-19487) Improve consistency of command-line arguments

2025-07-09 Thread Andrew Schofield (Jira)
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

[DISCUSS] KIP-1147: Improve consistency of command-line arguments

2025-07-09 Thread Andrew Schofield
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

Re: [DISCUSS] KIP-1172: Improve EndToEndLatency tool

2025-07-09 Thread Andrew Schofield
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

[jira] [Resolved] (KAFKA-19457) Revisit the threshold to retry initialize share state RPC in group coordinator.

2025-07-09 Thread Andrew Schofield (Jira)
[ 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

[jira] [Resolved] (KAFKA-19450) ShareConsumerPerformance does not handle exceptions from consumeMessagesForSingleShareConsumer

2025-07-09 Thread Andrew Schofield (Jira)
[ 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

[jira] [Resolved] (KAFKA-19463) nextFetchOffset does not take ongoing state transition into account

2025-07-08 Thread Andrew Schofield (Jira)
[ 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

[DISCUSS]: KIP-1191: Dead-letter queues for share groups

2025-07-08 Thread Andrew Schofield
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

[jira] [Created] (KAFKA-19471) Enable acknowledgement for a record which could not be deserialized

2025-07-04 Thread Andrew Schofield (Jira)
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

[jira] [Resolved] (KAFKA-19468) Share group epoch increments on every heartbeat if subscribed topics removed

2025-07-04 Thread Andrew Schofield (Jira)
[ 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

[jira] [Resolved] (KAFKA-19363) Enhance share group assignor for heterogeneous subscription type

2025-07-04 Thread Andrew Schofield (Jira)
[ 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

[jira] [Resolved] (KAFKA-19362) Enhance share group assignor for homogeneous subscription type

2025-07-04 Thread Andrew Schofield (Jira)
[ 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

[jira] [Created] (KAFKA-19468) Share group epoch increments on every heartbeat if subscribed topics removed

2025-07-03 Thread Andrew Schofield (Jira)
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

[jira] [Resolved] (KAFKA-19440) Admin.alterShareGroupOffsets doesn't handle exceptions correctly

2025-07-03 Thread Andrew Schofield (Jira)
[ 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

Re: [DISCUSS] Apache Kafka 4.1.0 release

2025-07-02 Thread Andrew Schofield
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

Re: [DISCUSS] Apache Kafka 4.1.0 release

2025-07-02 Thread Andrew Schofield
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

[jira] [Created] (KAFKA-19461) Add share group admin integration tests to PlaintextAdminIntegrationTest

2025-07-01 Thread Andrew Schofield (Jira)
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

[jira] [Created] (KAFKA-19440) Admin.alterShareGroupOffsets doesn't handle exceptions correctly

2025-06-26 Thread Andrew Schofield (Jira)
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

[jira] [Created] (KAFKA-19437) Review incidence of null metadata image in share coordinator

2025-06-26 Thread Andrew Schofield (Jira)
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

Re: [ANNOUNCE] New Kafka Committer: Omnia Ibrahim

2025-06-25 Thread Andrew Schofield
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.

Re: [VOTE] KIP-1160: Enable returning supported features from a specific broker

2025-06-25 Thread Andrew Schofield
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

[jira] [Resolved] (KAFKA-19416) Reduce running time of ShareConsumerTest

2025-06-19 Thread Andrew Schofield (Jira)
[ 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

[jira] [Created] (KAFKA-19416) Reduce running time of ShareConsumerTest

2025-06-18 Thread Andrew Schofield (Jira)
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

[jira] [Created] (KAFKA-19410) Validate alignment of subscribed topics and assigned partitions in share consumer

2025-06-16 Thread Andrew Schofield (Jira)
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

[jira] [Resolved] (KAFKA-19369) Add group.share.assignors config

2025-06-06 Thread Andrew Schofield (Jira)
[ 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

[jira] [Resolved] (KAFKA-19290) Exploit mapKey optimisation in protocol requests and responses

2025-06-06 Thread Andrew Schofield (Jira)
[ 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

[jira] [Resolved] (KAFKA-19370) Create JMH benchmark for assessing performance of share group assignor

2025-06-06 Thread Andrew Schofield (Jira)
[ 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

Re: [VOTE] KIP-1175: Fix the typo `PARTITIONER_ADPATIVE_PARTITIONING_ENABLE` in ProducerConfig

2025-06-05 Thread Andrew Schofield
+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日 週

[jira] [Resolved] (KAFKA-19328) SharePartitionManagerTest testMultipleConcurrentShareFetches doAnswer chaining needs verification

2025-06-04 Thread Andrew Schofield (Jira)
[ 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

[jira] [Created] (KAFKA-19370) Create JMH benchmark for assessing performance of share group assignor

2025-06-04 Thread Andrew Schofield (Jira)
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

[jira] [Created] (KAFKA-19369) Add group.share.assignors config

2025-06-04 Thread Andrew Schofield (Jira)
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

[jira] [Created] (KAFKA-19364) Production-ready documentation

2025-06-03 Thread Andrew Schofield (Jira)
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

[jira] [Created] (KAFKA-19363) Enhance share group assignor for heterogeneous subscription type

2025-06-03 Thread Andrew Schofield (Jira)
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

[jira] [Created] (KAFKA-19362) Enhance share group assignor for homogeneous subscription type

2025-06-03 Thread Andrew Schofield (Jira)
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

[jira] [Resolved] (KAFKA-19353) Preview documentation

2025-06-03 Thread Andrew Schofield (Jira)
[ 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

[jira] [Resolved] (KAFKA-19338) Share coordinator read state and write state should return error, if the share partition is not initialized.

2025-06-03 Thread Andrew Schofield (Jira)
[ 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

[jira] [Resolved] (KAFKA-19358) Do not update share group offset reset strategy from VerifiableShareConsumer in share_consumer_test.py tests

2025-06-02 Thread Andrew Schofield (Jira)
[ 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

[jira] [Resolved] (KAFKA-19320) Add share_consume_bench_test.py system tests

2025-06-02 Thread Andrew Schofield (Jira)
[ 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

[jira] [Resolved] (KAFKA-16894) Define switch to enable share groups for preview

2025-06-02 Thread Andrew Schofield (Jira)
[ 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

[jira] [Resolved] (KAFKA-19321) Add share_consumer_performance.py and related system tests

2025-06-02 Thread Andrew Schofield (Jira)
[ 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

[jira] [Created] (KAFKA-19353) Preview documentation

2025-05-30 Thread Andrew Schofield (Jira)
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

[jira] [Resolved] (KAFKA-19337) Bug in ShareCoordinatorShard.generateShareStateRecord causes stateEpoch update misses.

2025-05-29 Thread Andrew Schofield (Jira)
[ 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

[jira] [Resolved] (KAFKA-19316) Add share_group_command_test.py system test

2025-05-29 Thread Andrew Schofield (Jira)
[ 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

[jira] [Resolved] (KAFKA-19344) Move assignable share partition method from describer to group spec.

2025-05-28 Thread Andrew Schofield (Jira)
[ 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

[jira] [Resolved] (KAFKA-19342) Add AuthorizerIntegrationTest for alter share group offsets

2025-05-28 Thread Andrew Schofield (Jira)
[ 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

[jira] [Created] (KAFKA-19342) Add AuthorizerIntegrationTest for alter share group offsets

2025-05-27 Thread Andrew Schofield (Jira)
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

[jira] [Resolved] (KAFKA-19268) Missing mocks for SharePartitionManagerTest tests

2025-05-26 Thread Andrew Schofield (Jira)
[ 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

[jira] [Created] (KAFKA-19290) Exploit mapKey optimisation in protocol requests and responses

2025-05-16 Thread Andrew Schofield (Jira)
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

[jira] [Created] (KAFKA-19286) RemoteIndexCacheTest.testConcurrentRemoveReadForCache1 is flaky

2025-05-15 Thread Andrew Schofield (Jira)
Andrew Schofield created KAFKA-19286: Summary: RemoteIndexCacheTest.testConcurrentRemoveReadForCache1 is flaky Key: KAFKA-19286 URL: https://issues.apache.org/jira/browse/KAFKA-19286 Project

Re: KIP-1182 Quality of Service (QoS) for Apache Kafka

2025-05-14 Thread Andrew Schofield
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

Re: [DISCUSS] KIP-1178: Introduce remote.max.partition.fetch.bytes in Consumer

2025-05-13 Thread Andrew Schofield
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

[jira] [Resolved] (KAFKA-19201) Handle deletion of a topic which has initialised share-group state

2025-05-13 Thread Andrew Schofield (Jira)
[ 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

[jira] [Resolved] (KAFKA-19245) Update defaults for preview of queues configurations

2025-05-13 Thread Andrew Schofield (Jira)
[ 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

[jira] [Resolved] (KAFKA-19051) Fix implicit acknowledgement cannot be overridden when RecordDeserializationException occurs

2025-05-12 Thread Andrew Schofield (Jira)
[ 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

Re: [DISCUSS] KIP-1175: Fix the typo `PARTITIONER_ADPATIVE_PARTITIONING_ENABLE` in ProducerConfig

2025-05-08 Thread Andrew Schofield
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]

Re: [DISCUSS] KIP-1178: Introduce remote.max.partition.fetch.bytes in Consumer

2025-05-08 Thread Andrew Schofield
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

[jira] [Resolved] (KAFKA-19218) Add missing leader epoch to read share group state summary response

2025-05-06 Thread Andrew Schofield (Jira)
[ 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

[jira] [Resolved] (KAFKA-19216) Eliminate flakiness in kafka.server.share.SharePartitionTest

2025-05-05 Thread Andrew Schofield (Jira)
[ 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

[jira] [Resolved] (KAFKA-19015) Remove share session from cache on connection drop

2025-05-01 Thread Andrew Schofield (Jira)
[ 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

[jira] [Resolved] (KAFKA-19169) Enhance AuthorizerIntegrationTest for share group APIs

2025-05-01 Thread Andrew Schofield (Jira)
[ 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

[jira] [Resolved] (KAFKA-16718) Add AdminClient.deleteShareGroupOffsets

2025-04-30 Thread Andrew Schofield (Jira)
[ 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 >

[jira] [Created] (KAFKA-19218) Add missing leader epoch to read share group state summary response

2025-04-30 Thread Andrew Schofield (Jira)
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

[jira] [Created] (KAFKA-19217) ShareConsumerTest.testComplexShareConsumer is unreliable

2025-04-30 Thread Andrew Schofield (Jira)
Andrew Schofield created KAFKA-19217: Summary: ShareConsumerTest.testComplexShareConsumer is unreliable Key: KAFKA-19217 URL: https://issues.apache.org/jira/browse/KAFKA-19217 Project: Kafka

[jira] [Created] (KAFKA-19216) Eliminate flakiness in kafka.server.share.SharePartitionTest

2025-04-30 Thread Andrew Schofield (Jira)
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

Re: [VOTE] KIP-1123: Rack-aware partitioning for Kafka Producer

2025-04-29 Thread Andrew Schofield
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

Re: [DISCUSS] KIP-1043: Administration of groups

2025-04-29 Thread Andrew Schofield
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

[jira] [Created] (KAFKA-19204) Timeouts in coordinator runtime operations can break share group initialization and deletion

2025-04-25 Thread Andrew Schofield (Jira)
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

[jira] [Resolved] (KAFKA-19198) Deletion of topic assigned to a share group can result in NPE

2025-04-25 Thread Andrew Schofield (Jira)
[ 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

[jira] [Created] (KAFKA-19201) Handle deletion of a topic which has initialised share-group state

2025-04-25 Thread Andrew Schofield (Jira)
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

[jira] [Created] (KAFKA-19198) Deletion of topic assigned to a share group can result in NPE

2025-04-24 Thread Andrew Schofield (Jira)
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

[jira] [Created] (KAFKA-19189) Handle deletion of __share_group_state topic in share coordinator

2025-04-23 Thread Andrew Schofield (Jira)
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

[jira] [Resolved] (KAFKA-18629) DeleteShareGroups impl

2025-04-23 Thread Andrew Schofield (Jira)
[ 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

[jira] [Resolved] (KAFKA-19156) Remove group.share.max.groups config

2025-04-23 Thread Andrew Schofield (Jira)
[ 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

[jira] [Resolved] (KAFKA-18488) Additional protocol tests for share consumption

2025-04-23 Thread Andrew Schofield (Jira)
[ 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

[jira] [Resolved] (KAFKA-19057) Stabilize KIP-932 Kafka protocol RPC definitions in AK 4.1

2025-04-22 Thread Andrew Schofield (Jira)
[ 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

[jira] [Created] (KAFKA-19169) Enhance AuthorizerIntegrationTest for share group APIs

2025-04-17 Thread Andrew Schofield (Jira)
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

[jira] [Reopened] (KAFKA-19024) Enhance the client behaviour when it tries to exceed the `group.share.max.groups`

2025-04-17 Thread Andrew Schofield (Jira)
[ 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

[jira] [Resolved] (KAFKA-19024) Enhance the client behaviour when it tries to exceed the `group.share.max.groups`

2025-04-17 Thread Andrew Schofield (Jira)
[ 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

[jira] [Resolved] (KAFKA-19158) Add a new error code for “share session limit reached”

2025-04-16 Thread Andrew Schofield (Jira)
[ 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

Re: [VOTE] KIP-1101: Trigger rebalance on rack topology changes

2025-04-16 Thread Andrew Schofield
+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) >

[jira] [Resolved] (KAFKA-18902) Implement ShareConsumer option to throw on poll if there are unacked records

2025-04-15 Thread Andrew Schofield (Jira)
[ 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   2   3   4   5   6   7   >