Re: [PR] KAFKA-15481: Fix concurrency bug in RemoteIndexCache [kafka]

2023-10-05 Thread via GitHub
iit2009060 commented on code in PR #14483: URL: https://github.com/apache/kafka/pull/14483#discussion_r1348242651 ## core/src/test/scala/unit/kafka/log/remote/RemoteIndexCacheTest.scala: ## @@ -568,27 +621,26 @@ class RemoteIndexCacheTest { } private def

Re: [PR] KAFKA-15303: Avoid unnecessary re-serialization in FK-join [kafka]

2023-10-05 Thread via GitHub
mjsax commented on PR #14157: URL: https://github.com/apache/kafka/pull/14157#issuecomment-1749983456 @lucasbru -- I can play around and see if adding to `WrappedStore` would make sense instead of introducing a new interface. Not sure without playing with the code (not sure when I will get

Re: [PR] KAFKA-15552 Fix Producer ID ZK migration [kafka]

2023-10-05 Thread via GitHub
cmccabe merged PR #14506: URL: https://github.com/apache/kafka/pull/14506 -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail:

Re: [PR] KAFKA-15169: Added TestCase in RemoteIndexCache [kafka]

2023-10-05 Thread via GitHub
iit2009060 commented on PR #14482: URL: https://github.com/apache/kafka/pull/14482#issuecomment-1749979120 > @showuon @iit2009060 please don't wait for my review on this one. I might not get to it until next week. @showuon Can you review and address @divijvaidya comments. I have

Re: [PR] KAFKA-15417 flip joinSpuriousLookBackTimeMs and emit non-joined items [kafka]

2023-10-05 Thread via GitHub
mjsax commented on code in PR #14426: URL: https://github.com/apache/kafka/pull/14426#discussion_r1348237510 ## streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamOuterJoinTest.java: ## @@ -878,7 +878,9 @@ public void

Re: [PR] KAFKA-15417 flip joinSpuriousLookBackTimeMs and emit non-joined items [kafka]

2023-10-05 Thread via GitHub
mjsax commented on code in PR #14426: URL: https://github.com/apache/kafka/pull/14426#discussion_r1348236925 ## streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamOuterJoinTest.java: ## @@ -790,9 +790,7 @@ public void

Re: [PR] KAFKA-15417 flip joinSpuriousLookBackTimeMs and emit non-joined items [kafka]

2023-10-05 Thread via GitHub
mjsax commented on code in PR #14426: URL: https://github.com/apache/kafka/pull/14426#discussion_r1348235096 ## streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamLeftJoinTest.java: ## @@ -609,17 +842,18 @@ public void testOrdering() {

Re: [PR] KAFKA-15417 flip joinSpuriousLookBackTimeMs and emit non-joined items [kafka]

2023-10-05 Thread via GitHub
mjsax commented on PR #14426: URL: https://github.com/apache/kafka/pull/14426#issuecomment-1749965067 Thanks for the PR. I did not forget about it (sorry for the wait; very busy times...). > Moved the "emit non-joined items"-logic after the "joined items"-logic instead of before,

Re: [PR] KAFKA-15527: Add reverseRange and reverseAll query over kv-store in IQv2 [kafka]

2023-10-05 Thread via GitHub
mjsax commented on code in PR #14477: URL: https://github.com/apache/kafka/pull/14477#discussion_r1348228440 ## streams/src/test/java/org/apache/kafka/streams/integration/IQv2StoreIntegrationTest.java: ## @@ -816,29 +816,65 @@ private void shouldHandleRangeQueries(final

Re: [PR] KAFKA-15527: Add reverseRange and reverseAll query over kv-store in IQv2 [kafka]

2023-10-05 Thread via GitHub
mjsax commented on code in PR #14477: URL: https://github.com/apache/kafka/pull/14477#discussion_r1348227721 ## streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredKeyValueStore.java: ## @@ -254,17 +254,37 @@ protected QueryResult runRangeQuery(final Query

Re: [PR] KAFKA-15527: Add reverseRange and reverseAll query over kv-store in IQv2 [kafka]

2023-10-05 Thread via GitHub
mjsax commented on code in PR #14477: URL: https://github.com/apache/kafka/pull/14477#discussion_r1348226816 ## streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredKeyValueStore.java: ## @@ -254,17 +254,37 @@ protected QueryResult runRangeQuery(final Query

Re: [PR] KAFKA-14577: Move the scala ConsoleProducer from core to tools module [kafka]

2023-10-05 Thread via GitHub
github-actions[bot] commented on PR #13214: URL: https://github.com/apache/kafka/pull/13214#issuecomment-1749937548 This PR is being marked as stale since it has not had any activity in 90 days. If you would like to keep this PR alive, please ask a committer for review. If the PR has

Re: [PR] KAFKA-14987; Implement Group/Offset expiration in the new coordinator [kafka]

2023-10-05 Thread via GitHub
jeffkbkim commented on code in PR #14467: URL: https://github.com/apache/kafka/pull/14467#discussion_r1348205433 ## group-coordinator/src/main/java/org/apache/kafka/coordinator/group/OffsetMetadataManager.java: ## @@ -544,6 +573,81 @@ public

Re: [PR] KAFKA-14987; Implement Group/Offset expiration in the new coordinator [kafka]

2023-10-05 Thread via GitHub
jeffkbkim commented on code in PR #14467: URL: https://github.com/apache/kafka/pull/14467#discussion_r1348205086 ## group-coordinator/src/main/java/org/apache/kafka/coordinator/group/OffsetMetadataManager.java: ## @@ -544,6 +573,81 @@ public

Re: [PR] KAFKA-14987; Implement Group/Offset expiration in the new coordinator [kafka]

2023-10-05 Thread via GitHub
jeffkbkim commented on code in PR #14467: URL: https://github.com/apache/kafka/pull/14467#discussion_r1348199189 ## group-coordinator/src/main/java/org/apache/kafka/coordinator/group/generic/GenericGroup.java: ## @@ -898,6 +904,53 @@ public void createGroupTombstoneRecords(List

Re: [PR] KIP-951: Server side and protocol changes for KIP-951 [kafka]

2023-10-05 Thread via GitHub
jolshan commented on code in PR #1: URL: https://github.com/apache/kafka/pull/1#discussion_r1348136534 ## clients/src/main/resources/common/message/ProduceResponse.json: ## @@ -32,7 +32,9 @@ // records that cause the whole batch to be dropped. See KIP-467 for

Re: [PR] KAFKA-15355: Message schema changes [kafka]

2023-10-05 Thread via GitHub
soarez commented on code in PR #14290: URL: https://github.com/apache/kafka/pull/14290#discussion_r1348135503 ## generator/src/main/java/org/apache/kafka/message/FieldSpec.java: ## @@ -157,11 +157,6 @@ private void checkTagInvariants() { this.taggedVersions

Re: [PR] KIP-951: Server side and protocol changes for KIP-951 [kafka]

2023-10-05 Thread via GitHub
jolshan commented on code in PR #1: URL: https://github.com/apache/kafka/pull/1#discussion_r1348134556 ## clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java: ## @@ -67,20 +69,31 @@ public ProduceResponse(ProduceResponseData produceResponseData)

Re: [PR] KIP-951: Server side and protocol changes for KIP-951 [kafka]

2023-10-05 Thread via GitHub
jolshan commented on code in PR #1: URL: https://github.com/apache/kafka/pull/1#discussion_r1348129728 ## clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java: ## @@ -67,20 +69,31 @@ public ProduceResponse(ProduceResponseData produceResponseData)

Re: [PR] KIP-951: Server side and protocol changes for KIP-951 [kafka]

2023-10-05 Thread via GitHub
jolshan commented on code in PR #1: URL: https://github.com/apache/kafka/pull/1#discussion_r1348126229 ## clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java: ## @@ -65,24 +67,41 @@ public ProduceResponse(ProduceResponseData produceResponseData)

Re: [PR] KIP-951: Server side and protocol changes for KIP-951 [kafka]

2023-10-05 Thread via GitHub
jolshan commented on code in PR #1: URL: https://github.com/apache/kafka/pull/1#discussion_r1348126229 ## clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java: ## @@ -65,24 +67,41 @@ public ProduceResponse(ProduceResponseData produceResponseData)

Re: [PR] KIP-951: Server side and protocol changes for KIP-951 [kafka]

2023-10-05 Thread via GitHub
jolshan commented on code in PR #1: URL: https://github.com/apache/kafka/pull/1#discussion_r1348125403 ## clients/src/main/java/org/apache/kafka/common/requests/FetchResponse.java: ## @@ -276,11 +289,16 @@ private static FetchResponseData toMessage(Errors error,

Re: [PR] KIP-951: Server side and protocol changes for KIP-951 [kafka]

2023-10-05 Thread via GitHub
jolshan commented on code in PR #1: URL: https://github.com/apache/kafka/pull/1#discussion_r1348118773 ## clients/src/main/resources/common/message/FetchRequest.json: ## @@ -53,7 +53,9 @@ // // Version 15 adds the ReplicaState which includes new field ReplicaEpoch

[PR] KAFKA-15552 Fix Producer ID ZK migration [kafka]

2023-10-05 Thread via GitHub
mumrah opened a new pull request, #14506: URL: https://github.com/apache/kafka/pull/14506 This patch fixes a problem where we migrate the current producer ID batch to KRaft instead of the next producer ID batch. Since KRaft stores the _next_ batch in the log, we end up serving up a

Re: [PR] KAFKA-14519; [1/N] Implement coordinator runtime metrics [kafka]

2023-10-05 Thread via GitHub
jolshan commented on code in PR #14417: URL: https://github.com/apache/kafka/pull/14417#discussion_r1348107552 ## group-coordinator/src/test/java/org/apache/kafka/coordinator/group/runtime/CoordinatorRuntimeTest.java: ## @@ -1340,4 +1377,119 @@ public void

[jira] [Created] (KAFKA-15559) KIP-987: Connect Static Assignments

2023-10-05 Thread Greg Harris (Jira)
Greg Harris created KAFKA-15559: --- Summary: KIP-987: Connect Static Assignments Key: KAFKA-15559 URL: https://issues.apache.org/jira/browse/KAFKA-15559 Project: Kafka Issue Type: New Feature

Re: [PR] KAFKA-14274 [6, 7]: Introduction of fetch request manager [kafka]

2023-10-05 Thread via GitHub
kirktrue commented on code in PR #14406: URL: https://github.com/apache/kafka/pull/14406#discussion_r1348087535 ## clients/src/main/java/org/apache/kafka/clients/consumer/internals/PrototypeAsyncConsumer.java: ## @@ -636,42 +857,148 @@ public void assign(Collection partitions)

Re: [PR] KAFKA-14274 [6, 7]: Introduction of fetch request manager [kafka]

2023-10-05 Thread via GitHub
kirktrue commented on code in PR #14406: URL: https://github.com/apache/kafka/pull/14406#discussion_r1348084921 ## clients/src/main/java/org/apache/kafka/clients/consumer/internals/PrototypeAsyncConsumer.java: ## @@ -636,42 +857,148 @@ public void assign(Collection partitions)

Re: [PR] KAFKA-14519; [1/N] Implement coordinator runtime metrics [kafka]

2023-10-05 Thread via GitHub
jolshan commented on code in PR #14417: URL: https://github.com/apache/kafka/pull/14417#discussion_r1348084380 ## group-coordinator/src/test/java/org/apache/kafka/coordinator/group/metrics/GroupCoordinatorRuntimeMetricsTest.java: ## @@ -0,0 +1,237 @@ +/* + * Licensed to the

Re: [PR] KAFKA-14274 [6, 7]: Introduction of fetch request manager [kafka]

2023-10-05 Thread via GitHub
kirktrue commented on code in PR #14406: URL: https://github.com/apache/kafka/pull/14406#discussion_r1348083698 ## clients/src/main/java/org/apache/kafka/clients/consumer/internals/PrototypeAsyncConsumer.java: ## @@ -636,42 +857,148 @@ public void assign(Collection partitions)

Re: [PR] HOTIFX: fix Kafka versions for system tests [kafka]

2023-10-05 Thread via GitHub
mjsax commented on code in PR #14490: URL: https://github.com/apache/kafka/pull/14490#discussion_r1348082752 ## tests/kafkatest/version.py: ## @@ -120,7 +120,7 @@ def get_version(node=None): return DEV_BRANCH DEV_BRANCH = KafkaVersion("dev") -DEV_VERSION =

Re: [PR] HOTIFX: fix Kafka versions for system tests [kafka]

2023-10-05 Thread via GitHub
mjsax commented on code in PR #14490: URL: https://github.com/apache/kafka/pull/14490#discussion_r1348082752 ## tests/kafkatest/version.py: ## @@ -120,7 +120,7 @@ def get_version(node=None): return DEV_BRANCH DEV_BRANCH = KafkaVersion("dev") -DEV_VERSION =

Re: [PR] KAFKA-14734: Use CommandDefaultOptions in StreamsResetter [kafka]

2023-10-05 Thread via GitHub
mjsax commented on PR #13983: URL: https://github.com/apache/kafka/pull/13983#issuecomment-1749765712 Not a big deal -- as `StreamsResetter` is not public API, we don't need a deprecation period. I was just trying to say, because StreamsResetter is "semi-public" it might be nice to mention

[jira] [Resolved] (KAFKA-15437) Add metrics about open iterators

2023-10-05 Thread Matthias J. Sax (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15437?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax resolved KAFKA-15437. - Resolution: Duplicate > Add metrics about open iterators >

[jira] [Updated] (KAFKA-15539) Client should stop fetching while partitions being revoked

2023-10-05 Thread Yi Ding (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15539?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Yi Ding updated KAFKA-15539: Labels: kip-848 kip-848-client-support kip-848-preview (was: kip-848 kip-848-client-support) > Client

[jira] [Updated] (KAFKA-15325) Integrate topicId in OffsetFetch and OffsetCommit async consumer calls

2023-10-05 Thread Yi Ding (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15325?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Yi Ding updated KAFKA-15325: Labels: kip-848 kip-848-client-support kip-848-preview (was: kip-848 kip-848-client-support) >

[jira] [Updated] (KAFKA-15276) Implement partition assignment reconciliation

2023-10-05 Thread Yi Ding (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15276?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Yi Ding updated KAFKA-15276: Labels: kip-848 kip-848-client-support kip-848-preview (was: kip-848 kip-848-client-support) >

[jira] [Updated] (KAFKA-15321) Document consumer group member state machine

2023-10-05 Thread Yi Ding (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15321?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Yi Ding updated KAFKA-15321: Labels: kip-848 kip-848-client-support kip-848-preview (was: kip-848 kip-848-client-support) > Document

[jira] [Updated] (KAFKA-15280) Implement client support for KIP-848 server-side assignors

2023-10-05 Thread Yi Ding (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15280?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Yi Ding updated KAFKA-15280: Labels: consumer-threading-refactor kip-848 kip-848-client-support kip-848-preview (was:

[jira] [Updated] (KAFKA-15278) Implement client support for KIP-848 ConsumerGroupHeartbeat protocol RPC

2023-10-05 Thread Yi Ding (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15278?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Yi Ding updated KAFKA-15278: Labels: consumer-threading-refactor kip-848 kip-848-client-support kip-848-preview (was:

[jira] [Updated] (KAFKA-15544) Enable existing client integration tests for new protocol

2023-10-05 Thread Yi Ding (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15544?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Yi Ding updated KAFKA-15544: Labels: kip-848 kip-848-client-support kip-848-preview (was: kip-848 kip-848-client-support) > Enable

Re: [PR] KAFKA-14987; Implement Group/Offset expiration in the new coordinator [kafka]

2023-10-05 Thread via GitHub
jeffkbkim commented on code in PR #14467: URL: https://github.com/apache/kafka/pull/14467#discussion_r1348064529 ## group-coordinator/src/main/java/org/apache/kafka/coordinator/group/OffsetMetadataManager.java: ## @@ -544,6 +573,81 @@ public

[jira] [Updated] (KAFKA-15551) Evaluate conditions for short circuiting consumer API calls

2023-10-05 Thread Kirk True (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15551?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Kirk True updated KAFKA-15551: -- Description: For conditions like: * Committing empty offset * Fetching offsets for empty partitions

Re: [PR] KAFKA-14274 [6, 7]: Introduction of fetch request manager [kafka]

2023-10-05 Thread via GitHub
kirktrue commented on code in PR #14406: URL: https://github.com/apache/kafka/pull/14406#discussion_r1348062596 ## clients/src/main/java/org/apache/kafka/clients/consumer/internals/PrototypeAsyncConsumer.java: ## @@ -693,15 +1015,14 @@ private boolean

[jira] [Updated] (KAFKA-15558) Determine if Timer should be used elsewhere in PrototypeAsyncConsumer.updateFetchPositions()

2023-10-05 Thread Kirk True (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15558?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Kirk True updated KAFKA-15558: -- Description: This is a followup ticket based on a question from [~junrao] when reviewing the [fetch

Re: [PR] KAFKA-14274 [6, 7]: Introduction of fetch request manager [kafka]

2023-10-05 Thread via GitHub
kirktrue commented on code in PR #14406: URL: https://github.com/apache/kafka/pull/14406#discussion_r1348055761 ## clients/src/main/java/org/apache/kafka/clients/consumer/internals/PrototypeAsyncConsumer.java: ## @@ -621,56 +825,174 @@ public void assign(Collection partitions)

[jira] [Created] (KAFKA-15558) Determine if Timer should be used elsewhere in PrototypeAsyncConsumer.updateFetchPositions()

2023-10-05 Thread Kirk True (Jira)
Kirk True created KAFKA-15558: - Summary: Determine if Timer should be used elsewhere in PrototypeAsyncConsumer.updateFetchPositions() Key: KAFKA-15558 URL: https://issues.apache.org/jira/browse/KAFKA-15558

Re: [PR] KAFKA-14987; Implement Group/Offset expiration in the new coordinator [kafka]

2023-10-05 Thread via GitHub
jeffkbkim commented on code in PR #14467: URL: https://github.com/apache/kafka/pull/14467#discussion_r1348055071 ## group-coordinator/src/main/java/org/apache/kafka/coordinator/group/OffsetMetadataManager.java: ## @@ -544,6 +573,81 @@ public

Re: [PR] KAFKA-14274 [6, 7]: Introduction of fetch request manager [kafka]

2023-10-05 Thread via GitHub
kirktrue commented on code in PR #14406: URL: https://github.com/apache/kafka/pull/14406#discussion_r1348047927 ## clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchRequestManagerTest.java: ## @@ -0,0 +1,3565 @@ +/* + * Licensed to the Apache Software

[jira] [Updated] (KAFKA-15557) Fix duplicate metadata update in fetcher tests

2023-10-05 Thread Kirk True (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15557?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Kirk True updated KAFKA-15557: -- Issue Type: Test (was: Bug) > Fix duplicate metadata update in fetcher tests >

[jira] [Created] (KAFKA-15557) Fix duplicate metadata update in fetcher tests

2023-10-05 Thread Kirk True (Jira)
Kirk True created KAFKA-15557: - Summary: Fix duplicate metadata update in fetcher tests Key: KAFKA-15557 URL: https://issues.apache.org/jira/browse/KAFKA-15557 Project: Kafka Issue Type: Bug

[jira] [Commented] (KAFKA-15548) Handling close() properly

2023-10-05 Thread Kirk True (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15548?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17772396#comment-17772396 ] Kirk True commented on KAFKA-15548: --- [~lianetm] I think KAFKA-15540 is only related to item #3 in the

[jira] [Updated] (KAFKA-15548) Handling close() properly

2023-10-05 Thread Kirk True (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15548?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Kirk True updated KAFKA-15548: -- Description: Upon closing of the {{Consumer}} we need to: # Complete pending commits # Auto-commit

Re: [PR] KAFKA-14274 [6, 7]: Introduction of fetch request manager [kafka]

2023-10-05 Thread via GitHub
kirktrue commented on code in PR #14406: URL: https://github.com/apache/kafka/pull/14406#discussion_r1346606932 ## clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java: ## @@ -106,7 +127,62 @@ public void onFailure(RuntimeException e) { return

Re: [PR] KAFKA-14274 [6, 7]: Introduction of fetch request manager [kafka]

2023-10-05 Thread via GitHub
kirktrue commented on code in PR #14406: URL: https://github.com/apache/kafka/pull/14406#discussion_r1348029671 ## clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractFetch.java: ## @@ -330,7 +343,7 @@ private Map prepareCloseFetchSession

Re: [PR] KAFKA-14274 [6, 7]: Introduction of fetch request manager [kafka]

2023-10-05 Thread via GitHub
kirktrue commented on code in PR #14406: URL: https://github.com/apache/kafka/pull/14406#discussion_r1348029408 ## clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractFetch.java: ## @@ -92,6 +88,23 @@ public AbstractFetch(final LogContext logContext,

[jira] [Updated] (KAFKA-15556) Remove NetworkClientDelegate methods isUnavailable, maybeThrowAuthFailure, and tryConnect

2023-10-05 Thread Kirk True (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15556?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Kirk True updated KAFKA-15556: -- Summary: Remove NetworkClientDelegate methods isUnavailable, maybeThrowAuthFailure, and tryConnect

[jira] [Created] (KAFKA-15556) Remove isUnavailable and maybeThrowAuthFailure if possible

2023-10-05 Thread Kirk True (Jira)
Kirk True created KAFKA-15556: - Summary: Remove isUnavailable and maybeThrowAuthFailure if possible Key: KAFKA-15556 URL: https://issues.apache.org/jira/browse/KAFKA-15556 Project: Kafka Issue

[jira] [Commented] (KAFKA-15304) CompletableApplicationEvents aren't being completed when the consumer is closing

2023-10-05 Thread Kirk True (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15304?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17772394#comment-17772394 ] Kirk True commented on KAFKA-15304: --- Didn't you fix this, [~pnee]? > CompletableApplicationEvents

Re: [PR] KAFKA-14274 [6, 7]: Introduction of fetch request manager [kafka]

2023-10-05 Thread via GitHub
kirktrue commented on code in PR #14406: URL: https://github.com/apache/kafka/pull/14406#discussion_r1348011353 ## clients/src/main/java/org/apache/kafka/clients/consumer/internals/PrototypeAsyncConsumer.java: ## @@ -192,71 +321,38 @@ public PrototypeAsyncConsumer(final

Re: [PR] KAFKA-14274 [6, 7]: Introduction of fetch request manager [kafka]

2023-10-05 Thread via GitHub
kirktrue commented on code in PR #14406: URL: https://github.com/apache/kafka/pull/14406#discussion_r1348009622 ## clients/src/main/java/org/apache/kafka/clients/consumer/internals/PrototypeAsyncConsumer.java: ## @@ -192,71 +321,38 @@ public PrototypeAsyncConsumer(final

Re: [PR] KAFKA-14274 [6, 7]: Introduction of fetch request manager [kafka]

2023-10-05 Thread via GitHub
kirktrue commented on code in PR #14406: URL: https://github.com/apache/kafka/pull/14406#discussion_r1348009426 ## clients/src/main/java/org/apache/kafka/clients/consumer/internals/PrototypeAsyncConsumer.java: ## @@ -192,71 +321,38 @@ public PrototypeAsyncConsumer(final

[jira] [Created] (KAFKA-15555) Ensure wakeups are handled correctly in PrototypeAsyncConsumer.poll()

2023-10-05 Thread Kirk True (Jira)
Kirk True created KAFKA-1: - Summary: Ensure wakeups are handled correctly in PrototypeAsyncConsumer.poll() Key: KAFKA-1 URL: https://issues.apache.org/jira/browse/KAFKA-1 Project: Kafka

Re: [PR] KAFKA-14519; [1/N] Implement coordinator runtime metrics [kafka]

2023-10-05 Thread via GitHub
jolshan commented on code in PR #14417: URL: https://github.com/apache/kafka/pull/14417#discussion_r1348006202 ## group-coordinator/src/main/java/org/apache/kafka/coordinator/group/metrics/CoordinatorRuntimeMetrics.java: ## @@ -0,0 +1,68 @@ +/* + * Licensed to the Apache

[jira] [Updated] (KAFKA-15508) Method always return the same value ApplicationEventProcessor.java

2023-10-05 Thread Kirk True (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15508?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Kirk True updated KAFKA-15508: -- Parent: KAFKA-14246 Issue Type: Sub-task (was: Bug) > Method always return the same value

[jira] [Assigned] (KAFKA-15508) Method always return the same value ApplicationEventProcessor.java

2023-10-05 Thread Kirk True (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15508?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Kirk True reassigned KAFKA-15508: - Assignee: Kirk True > Method always return the same value ApplicationEventProcessor.java >

Re: [PR] KAFKA-14519; [1/N] Implement coordinator runtime metrics [kafka]

2023-10-05 Thread via GitHub
jolshan commented on code in PR #14417: URL: https://github.com/apache/kafka/pull/14417#discussion_r1348003447 ## group-coordinator/src/main/java/org/apache/kafka/coordinator/group/metrics/GroupCoordinatorRuntimeMetrics.java: ## @@ -0,0 +1,266 @@ +/* + * Licensed to the Apache

Re: [PR] KAFKA-14519; [1/N] Implement coordinator runtime metrics [kafka]

2023-10-05 Thread via GitHub
jolshan commented on code in PR #14417: URL: https://github.com/apache/kafka/pull/14417#discussion_r1347999373 ## group-coordinator/src/main/java/org/apache/kafka/coordinator/group/metrics/GroupCoordinatorRuntimeMetrics.java: ## @@ -0,0 +1,266 @@ +/* + * Licensed to the Apache

Re: [PR] KAFKA-14519; [1/N] Implement coordinator runtime metrics [kafka]

2023-10-05 Thread via GitHub
jolshan commented on code in PR #14417: URL: https://github.com/apache/kafka/pull/14417#discussion_r1348000674 ## group-coordinator/src/main/java/org/apache/kafka/coordinator/group/metrics/GroupCoordinatorRuntimeMetrics.java: ## @@ -0,0 +1,266 @@ +/* + * Licensed to the Apache

Re: [PR] KAFKA-14519; [1/N] Implement coordinator runtime metrics [kafka]

2023-10-05 Thread via GitHub
jolshan commented on code in PR #14417: URL: https://github.com/apache/kafka/pull/14417#discussion_r1347998253 ## group-coordinator/src/main/java/org/apache/kafka/coordinator/group/metrics/CoordinatorRuntimeMetrics.java: ## @@ -0,0 +1,100 @@ +/* + * Licensed to the Apache

Re: [PR] KAFKA-14519; [1/N] Implement coordinator runtime metrics [kafka]

2023-10-05 Thread via GitHub
jolshan commented on code in PR #14417: URL: https://github.com/apache/kafka/pull/14417#discussion_r1347989806 ## core/src/main/scala/kafka/coordinator/group/CoordinatorLoaderImpl.scala: ## @@ -143,10 +150,12 @@ class CoordinatorLoaderImpl[T]( currentOffset =

[jira] [Commented] (KAFKA-15552) Duplicate Producer ID blocks during ZK migration

2023-10-05 Thread Justine Olshan (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15552?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17772375#comment-17772375 ] Justine Olshan commented on KAFKA-15552: Recovering when this happens on idempotent producers

Re: [PR] KAFKA-14274 [6, 7]: Introduction of fetch request manager [kafka]

2023-10-05 Thread via GitHub
lianetm commented on code in PR #14406: URL: https://github.com/apache/kafka/pull/14406#discussion_r1347752851 ## clients/src/main/java/org/apache/kafka/clients/consumer/internals/PrototypeAsyncConsumer.java: ## @@ -621,56 +825,174 @@ public void assign(Collection partitions)

[jira] [Created] (KAFKA-15554) Update client state machine to align with protocol sending one assignment at a time

2023-10-05 Thread Lianet Magrans (Jira)
Lianet Magrans created KAFKA-15554: -- Summary: Update client state machine to align with protocol sending one assignment at a time Key: KAFKA-15554 URL: https://issues.apache.org/jira/browse/KAFKA-15554

[jira] [Resolved] (KAFKA-15275) Implement consumer group membership state machine

2023-10-05 Thread Lianet Magrans (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15275?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Lianet Magrans resolved KAFKA-15275. Resolution: Fixed > Implement consumer group membership state machine >

[jira] [Updated] (KAFKA-15275) Implement consumer group membership state machine

2023-10-05 Thread Lianet Magrans (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15275?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Lianet Magrans updated KAFKA-15275: --- Description: Provide the Java client support for the consumer group member state machine,

Re: [PR] KAFKA-14684 Replace EasyMock and PowerMock with Mockito in WorkerSinkTaskThreadedTest [kafka]

2023-10-05 Thread via GitHub
hgeraldino commented on PR #14505: URL: https://github.com/apache/kafka/pull/14505#issuecomment-1749547468 Hey @C0urante @gharris1727 @yashmayya @clolov I was pleasantly surprised to find out that **only 4!** of the connect:runtime tests are pending to be migrated. Here's my attempt

[PR] KAFKA-14684 Replace EasyMock and PowerMock with Mockito in WorkerSinkTaskThreadedTest [kafka]

2023-10-05 Thread via GitHub
hgeraldino opened a new pull request, #14505: URL: https://github.com/apache/kafka/pull/14505 JIRA: https://issues.apache.org/jira/browse/KAFKA-14684 Previous [PR](https://github.com/apache/kafka/pull/13951) got closed somehow *More detailed description of your change, if

Re: [PR] KAFKA-14274 [6, 7]: Introduction of fetch request manager [kafka]

2023-10-05 Thread via GitHub
lianetm commented on code in PR #14406: URL: https://github.com/apache/kafka/pull/14406#discussion_r1347885711 ## clients/src/main/java/org/apache/kafka/clients/consumer/internals/PrototypeAsyncConsumer.java: ## @@ -693,15 +1015,14 @@ private boolean

Re: [PR] KAFKA-14684 Replace EasyMock and PowerMock with Mockito in WorkerSinkTaskThreadedTest [kafka]

2023-10-05 Thread via GitHub
hgeraldino closed pull request #13951: KAFKA-14684 Replace EasyMock and PowerMock with Mockito in WorkerSinkTaskThreadedTest URL: https://github.com/apache/kafka/pull/13951 -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use

[jira] [Updated] (KAFKA-15553) Review committed offset refresh logic

2023-10-05 Thread Philip Nee (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15553?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Philip Nee updated KAFKA-15553: --- Labels: consumer-threading-refactor (was: ) > Review committed offset refresh logic >

[jira] [Updated] (KAFKA-15553) Review committed offset refresh logic

2023-10-05 Thread Philip Nee (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15553?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Philip Nee updated KAFKA-15553: --- Parent: KAFKA-14246 Issue Type: Sub-task (was: Bug) > Review committed offset refresh

[jira] [Created] (KAFKA-15553) Review committed offset refresh logic

2023-10-05 Thread Philip Nee (Jira)
Philip Nee created KAFKA-15553: -- Summary: Review committed offset refresh logic Key: KAFKA-15553 URL: https://issues.apache.org/jira/browse/KAFKA-15553 Project: Kafka Issue Type: Bug

Re: [PR] KAFKA-14274 [6, 7]: Introduction of fetch request manager [kafka]

2023-10-05 Thread via GitHub
kirktrue commented on code in PR #14406: URL: https://github.com/apache/kafka/pull/14406#discussion_r1347874693 ## clients/src/main/java/org/apache/kafka/clients/consumer/internals/PrototypeAsyncConsumer.java: ## @@ -621,56 +825,174 @@ public void assign(Collection partitions)

Re: [PR] KAFKA-14274 [6, 7]: Introduction of fetch request manager [kafka]

2023-10-05 Thread via GitHub
kirktrue commented on code in PR #14406: URL: https://github.com/apache/kafka/pull/14406#discussion_r1347874026 ## clients/src/main/java/org/apache/kafka/clients/consumer/internals/PrototypeAsyncConsumer.java: ## @@ -693,15 +1015,14 @@ private boolean

[jira] [Created] (KAFKA-15552) Duplicate Producer ID blocks during ZK migration

2023-10-05 Thread David Arthur (Jira)
David Arthur created KAFKA-15552: Summary: Duplicate Producer ID blocks during ZK migration Key: KAFKA-15552 URL: https://issues.apache.org/jira/browse/KAFKA-15552 Project: Kafka Issue Type:

Re: [PR] KIP-951: Server side and protocol changes for KIP-951 [kafka]

2023-10-05 Thread via GitHub
chb2ab commented on code in PR #1: URL: https://github.com/apache/kafka/pull/1#discussion_r1347868792 ## clients/src/main/resources/common/message/ProduceResponse.json: ## @@ -32,7 +32,9 @@ // records that cause the whole batch to be dropped. See KIP-467 for

Re: [PR] Kafka 12317: Relax non-null key requirement in Kafka Streams [kafka]

2023-10-05 Thread via GitHub
florin-akermann commented on PR #14174: URL: https://github.com/apache/kafka/pull/14174#issuecomment-1749481836 Hey @wcarlson5 Great! Yes I am willing to push it. Looking forward to your feedback. -- This is an automated message from the Apache Git Service. To respond to the

Re: [PR] Kafka 12317: Relax non-null key requirement in Kafka Streams [kafka]

2023-10-05 Thread via GitHub
wcarlson5 commented on PR #14174: URL: https://github.com/apache/kafka/pull/14174#issuecomment-1749460142 Hey @florin-akermann I can take a look at this soon. I'll probably be able to take a first look next week if you are still willing to push this -- This is an automated message from

[jira] [Updated] (KAFKA-15531) Ensure coordinator node is removed upon disconnection exception

2023-10-05 Thread Philip Nee (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15531?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Philip Nee updated KAFKA-15531: --- Labels: kip-848 kip-848-client-support kip-848-preview (was: kip-848 kip-848-client-support) >

[jira] [Updated] (KAFKA-15533) Ensure HeartbeatRequestManager only send out some fields once

2023-10-05 Thread Philip Nee (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15533?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Philip Nee updated KAFKA-15533: --- Labels: kip-848-preview (was: ) > Ensure HeartbeatRequestManager only send out some fields once >

[jira] [Updated] (KAFKA-15551) Evaluate conditions for short circuiting consumer API calls

2023-10-05 Thread Philip Nee (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15551?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Philip Nee updated KAFKA-15551: --- Parent: KAFKA-14246 Issue Type: Sub-task (was: Bug) > Evaluate conditions for short

[jira] [Created] (KAFKA-15551) Evaluate conditions for short circuiting consumer API calls

2023-10-05 Thread Philip Nee (Jira)
Philip Nee created KAFKA-15551: -- Summary: Evaluate conditions for short circuiting consumer API calls Key: KAFKA-15551 URL: https://issues.apache.org/jira/browse/KAFKA-15551 Project: Kafka

Re: [PR] KAFKA-14274 [6, 7]: Introduction of fetch request manager [kafka]

2023-10-05 Thread via GitHub
kirktrue commented on code in PR #14406: URL: https://github.com/apache/kafka/pull/14406#discussion_r1347799078 ## clients/src/main/java/org/apache/kafka/clients/consumer/internals/PrototypeAsyncConsumer.java: ## @@ -192,71 +321,38 @@ public PrototypeAsyncConsumer(final

[jira] [Comment Edited] (KAFKA-15452) Custom KafkaPrincipalBuilder Cannot Access SslPrincipalMapper

2023-10-05 Thread Raghu Baddam (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15452?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17765305#comment-17765305 ] Raghu Baddam edited comment on KAFKA-15452 at 10/5/23 6:01 PM: --- Thanks

Re: [PR] MINOR: update Kafka versions for system tests [kafka]

2023-10-05 Thread via GitHub
mjsax merged PR #14501: URL: https://github.com/apache/kafka/pull/14501 -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail:

Re: [PR] HOTIFX: fix Kafka versions for system tests [kafka]

2023-10-05 Thread via GitHub
mjsax merged PR #14500: URL: https://github.com/apache/kafka/pull/14500 -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail:

Re: [PR] MINOR: upgrade Kafka versions for system tests [kafka]

2023-10-05 Thread via GitHub
mjsax merged PR #14499: URL: https://github.com/apache/kafka/pull/14499 -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail:

Re: [PR] HOTIFX: fix Kafka versions for system tests [kafka]

2023-10-05 Thread via GitHub
mjsax merged PR #14498: URL: https://github.com/apache/kafka/pull/14498 -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail:

Re: [PR] HOTIFX: fix Kafka versions for system tests [kafka]

2023-10-05 Thread via GitHub
mjsax merged PR #14497: URL: https://github.com/apache/kafka/pull/14497 -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail:

  1   2   3   >