[jira] [Assigned] (KAFKA-16101) KRaft migration documentation is incorrect

2024-01-15 Thread Colin McCabe (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16101?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Colin McCabe reassigned KAFKA-16101: Fix Version/s: 3.7.0 Assignee: Colin McCabe Priority: Blocker (was:

[jira] [Commented] (KAFKA-16101) KRaft migration documentation is incorrect

2024-01-15 Thread Colin McCabe (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16101?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17806678#comment-17806678 ] Colin McCabe commented on KAFKA-16101: -- Hi Luke, Thanks for testing rollback. I think this is a

Re: [PR] KAFKA-16101: KRaft migration documentation is incorrect [kafka]

2024-01-15 Thread via GitHub
ppatierno commented on code in PR #15193: URL: https://github.com/apache/kafka/pull/15193#discussion_r1452070809 ## docs/ops.html: ## @@ -3956,25 +3956,31 @@ Reverting to ZooKeeper mode During the Migration While the cluster is still in migration mode, it is possible to

Re: [PR] KAFKA-13922: Adjustments for jacoco, coverage reporting [kafka]

2024-01-15 Thread via GitHub
akatona84 commented on code in PR #11982: URL: https://github.com/apache/kafka/pull/11982#discussion_r1452076677 ## build.gradle: ## @@ -998,6 +973,9 @@ project(':core') { if (userEnableTestCoverage) { scoverage { scoverageVersion = versions.scoverage + if

[PR] KAFKA-16101: KRaft migration documentation is incorrect [kafka]

2024-01-15 Thread via GitHub
cmccabe opened a new pull request, #15193: URL: https://github.com/apache/kafka/pull/15193 The documentation about reverting from KRaft mode to ZK mode previously said that the admin should roll each broker to remove the migration configuration, then take down the kcontrollers. However,

[jira] [Updated] (KAFKA-16101) KRaft migration documentation is incorrect

2024-01-15 Thread Colin McCabe (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16101?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Colin McCabe updated KAFKA-16101: - Summary: KRaft migration documentation is incorrect (was: Kafka cluster unavailable during

[jira] [Created] (KAFKA-16130) Test migration rollback

2024-01-15 Thread Mickael Maison (Jira)
Mickael Maison created KAFKA-16130: -- Summary: Test migration rollback Key: KAFKA-16130 URL: https://issues.apache.org/jira/browse/KAFKA-16130 Project: Kafka Issue Type: Sub-task

Re: [PR] KAFKA-16101: KRaft migration documentation is incorrect [kafka]

2024-01-15 Thread via GitHub
showuon commented on PR #15193: URL: https://github.com/apache/kafka/pull/15193#issuecomment-1891547221 From @ppatierno ``` Hi Colin, I was the one raising the issue about rollback and I also already tried what you mentioned but with no success. During the first rolling,

[jira] [Commented] (KAFKA-16101) KRaft migration documentation is incorrect

2024-01-15 Thread Mickael Maison (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16101?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17806706#comment-17806706 ] Mickael Maison commented on KAFKA-16101: Updating the docs is a good first step but we should

[jira] [Commented] (KAFKA-16132) Upgrading from 3.6 to 3.7 in KRaft will have seconds of partitions unavailable

2024-01-15 Thread Luke Chen (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16132?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17806799#comment-17806799 ] Luke Chen commented on KAFKA-16132: --- [~cmccabe] [~soarez] , please take a look. Thanks. > Upgrading

Re: [PR] KAFKA-15942: Implement ConsumerInterceptors in the async consumer [kafka]

2024-01-15 Thread via GitHub
lucasbru commented on PR #15000: URL: https://github.com/apache/kafka/pull/15000#issuecomment-1892152097 > This PR looks about ready to merge. Please can you change `PlaintextConsumerTest` to run the tests for interceptors with the new consumer. They should pass once this PR is merged.

Re: [PR] KAFKA-15942: Implement ConsumerInterceptors in the async consumer [kafka]

2024-01-15 Thread via GitHub
lucasbru commented on code in PR #15000: URL: https://github.com/apache/kafka/pull/15000#discussion_r1452363674 ## clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java: ## @@ -1226,6 +1236,8 @@ private void close(Duration timeout, boolean

[jira] [Created] (KAFKA-16132) Upgrading from 3.6 to 3.7 in KRaft will have seconds of partitions unavailable

2024-01-15 Thread Luke Chen (Jira)
Luke Chen created KAFKA-16132: - Summary: Upgrading from 3.6 to 3.7 in KRaft will have seconds of partitions unavailable Key: KAFKA-16132 URL: https://issues.apache.org/jira/browse/KAFKA-16132 Project:

[jira] [Commented] (KAFKA-16131) Repeated UnsupportedVersionException logged when running Kafka 3.7.0-RC2 KRaft cluster with metadata version 3.6

2024-01-15 Thread Luke Chen (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16131?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17806800#comment-17806800 ] Luke Chen commented on KAFKA-16131: --- KAFKA-16132 is also a bug related to JBOD after upgrading to 3.7.

[jira] [Created] (KAFKA-16133) Commits during reconciliation always time out

2024-01-15 Thread Lucas Brutschy (Jira)
Lucas Brutschy created KAFKA-16133: -- Summary: Commits during reconciliation always time out Key: KAFKA-16133 URL: https://issues.apache.org/jira/browse/KAFKA-16133 Project: Kafka Issue

Re: [PR] KAFKA-15942: Implement ConsumerInterceptors in the async consumer [kafka]

2024-01-15 Thread via GitHub
lucasbru commented on PR #15000: URL: https://github.com/apache/kafka/pull/15000#issuecomment-1892152810 @cadonna Can you please review this? -- 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

[jira] [Updated] (KAFKA-16131) Repeated UnsupportedVersionException logged when running Kafka 3.7.0-RC2 KRaft cluster with metadata version 3.6

2024-01-15 Thread Ismael Juma (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16131?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-16131: Priority: Blocker (was: Major) > Repeated UnsupportedVersionException logged when running Kafka

Re: [PR] KAFKA-15807: Added support for compression of metrics (KIP-714) [kafka]

2024-01-15 Thread via GitHub
apoorvmittal10 commented on code in PR #15148: URL: https://github.com/apache/kafka/pull/15148#discussion_r1452404874 ## clients/src/main/java/org/apache/kafka/common/telemetry/internals/ClientTelemetryReporter.java: ## @@ -715,15 +716,22 @@ private Optional>

Re: [PR] [Kafka-14404] fix overlap of streams-config sections & describe additional parameters [kafka]

2024-01-15 Thread via GitHub
AyoubOm commented on PR #15162: URL: https://github.com/apache/kafka/pull/15162#issuecomment-1891809951 > Thanks for the PR. Couple of comments. Thanks @mjsax for the review. Please check the changes pushed -- This is an automated message from the Apache Git Service. To respond to

Re: [PR] [Kafka-14404] fix overlap of streams-config sections & describe additional parameters [kafka]

2024-01-15 Thread via GitHub
AyoubOm commented on code in PR #15162: URL: https://github.com/apache/kafka/pull/15162#discussion_r1452182928 ## docs/streams/developer-guide/config-streams.html: ## @@ -,28 +,31 @@ Default ValuesConsumer earliest -linger.ms +

Re: [PR] [Kafka-14404] fix overlap of streams-config sections & describe additional parameters [kafka]

2024-01-15 Thread via GitHub
AyoubOm commented on code in PR #15162: URL: https://github.com/apache/kafka/pull/15162#discussion_r1452182174 ## docs/streams/developer-guide/config-streams.html: ## @@ -,28 +,31 @@ Default ValuesConsumer earliest -linger.ms +

[jira] [Commented] (KAFKA-14304) ZooKeeper to KRaft Migration

2024-01-15 Thread Mickael Maison (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-14304?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17806713#comment-17806713 ] Mickael Maison commented on KAFKA-14304: I created

[jira] [Created] (KAFKA-16131) Repeated UnsupportedVersionException logged when running Kafka 3.7.0-RC2 KRaft cluster with metadata version 3.6

2024-01-15 Thread Jakub Scholz (Jira)
Jakub Scholz created KAFKA-16131: Summary: Repeated UnsupportedVersionException logged when running Kafka 3.7.0-RC2 KRaft cluster with metadata version 3.6 Key: KAFKA-16131 URL:

Re: [PR] [Kafka-14404] fix overlap of streams-config sections & describe additional parameters [kafka]

2024-01-15 Thread via GitHub
AyoubOm commented on code in PR #15162: URL: https://github.com/apache/kafka/pull/15162#discussion_r1452186660 ## docs/streams/developer-guide/config-streams.html: ## @@ -,28 +,31 @@ Default ValuesConsumer earliest -linger.ms +

Re: [PR] [Kafka-14404] fix overlap of streams-config sections & describe additional parameters [kafka]

2024-01-15 Thread via GitHub
AyoubOm commented on code in PR #15162: URL: https://github.com/apache/kafka/pull/15162#discussion_r1452182174 ## docs/streams/developer-guide/config-streams.html: ## @@ -,28 +,31 @@ Default ValuesConsumer earliest -linger.ms +

[jira] [Resolved] (KAFKA-15740) KRaft support in DeleteOffsetsConsumerGroupCommandIntegrationTest

2024-01-15 Thread Mickael Maison (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15740?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Mickael Maison resolved KAFKA-15740. Fix Version/s: 3.8.0 Resolution: Fixed > KRaft support in

Re: [PR] KAFKA-15740: KRaft support in DeleteOffsetsConsumerGroupCommandIntegrationTest [kafka]

2024-01-15 Thread via GitHub
mimaison merged PR #14669: URL: https://github.com/apache/kafka/pull/14669 -- 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-15942: Implement ConsumerInterceptor [kafka]

2024-01-15 Thread via GitHub
lucasbru commented on PR #14963: URL: https://github.com/apache/kafka/pull/14963#issuecomment-1891690106 @Joker-5 I can add it in the PR description so that I don't forget. But the tag needs to added to the final commit, once we merge it, so we cannot really add it yet -- This is an

[jira] [Updated] (KAFKA-16073) Kafka Tiered Storage: Consumer Fetch Error Due to Delayed localLogStartOffset Update During Segment Deletion

2024-01-15 Thread Satish Duggana (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16073?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Satish Duggana updated KAFKA-16073: --- Summary: Kafka Tiered Storage: Consumer Fetch Error Due to Delayed localLogStartOffset

Re: [PR] [Kafka-14404] fix overlap of streams-config sections & describe additional parameters [kafka]

2024-01-15 Thread via GitHub
AyoubOm commented on code in PR #15162: URL: https://github.com/apache/kafka/pull/15162#discussion_r1452180948 ## docs/streams/developer-guide/config-streams.html: ## @@ -,28 +,31 @@ Default ValuesConsumer Review Comment: Version fixed. For `max.poll.records` it

Re: [PR] [Kafka-14404] fix overlap of streams-config sections & describe additional parameters [kafka]

2024-01-15 Thread via GitHub
AyoubOm commented on code in PR #15162: URL: https://github.com/apache/kafka/pull/15162#discussion_r1452187247 ## docs/streams/developer-guide/config-streams.html: ## @@ -409,12 +409,12 @@ num.standby.replicasThe replication factor for changelog topics and repartition topics

Re: [PR] KAFKA-15750: KRaft support in KafkaMetricReporterExceptionHandlingTest [kafka]

2024-01-15 Thread via GitHub
mimaison merged PR #14707: URL: https://github.com/apache/kafka/pull/14707 -- 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-16029: Fix "Unable to find FetchSessionHandler for node X" bug [kafka]

2024-01-15 Thread via GitHub
ijuma commented on PR #15186: URL: https://github.com/apache/kafka/pull/15186#issuecomment-1892830178 Are there any network resources held by the `sessionHandlers` that would no longer be closed due to this? If not, then this seems like a reasonable fix. -- This is an automated message

Re: [PR] KAFKA-13988: Fix MM2 not consuming from latest when "auto.offset.reset=latest" is set [kafka]

2024-01-15 Thread via GitHub
github-actions[bot] commented on PR #13905: URL: https://github.com/apache/kafka/pull/13905#issuecomment-1893019795 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

[jira] [Updated] (KAFKA-16135) kafka.api.PlaintextConsumerTest.testPerPartitionLeadMetricsCleanUpWithSubscribe(String, String).quorum=kraft+kip848.groupProtocol=consumer is flaky

2024-01-15 Thread Kirk True (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16135?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Kirk True updated KAFKA-16135: -- Component/s: clients consumer >

[jira] [Updated] (KAFKA-16135) kafka.api.PlaintextConsumerTest.testPerPartitionLeadMetricsCleanUpWithSubscribe(String, String).quorum=kraft+kip848.groupProtocol=consumer is flaky

2024-01-15 Thread Kirk True (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16135?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Kirk True updated KAFKA-16135: -- Labels: consumer-threading-refactor kip-848-client-support (was: ) >

[jira] [Updated] (KAFKA-16134) kafka.api.PlaintextConsumerTest.testPerPartitionLagMetricsCleanUpWithSubscribe(String, String).quorum=kraft+kip848.groupProtocol=consumer is flaky

2024-01-15 Thread Kirk True (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16134?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Kirk True updated KAFKA-16134: -- Fix Version/s: 3.8.0 >

[jira] [Assigned] (KAFKA-16142) Update metrics documentation for errors and new metrics

2024-01-15 Thread Kirk True (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16142?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Kirk True reassigned KAFKA-16142: - Assignee: Philip Nee > Update metrics documentation for errors and new metrics >

[jira] [Created] (KAFKA-16142) Update metrics documentation for errors and new metrics

2024-01-15 Thread Kirk True (Jira)
Kirk True created KAFKA-16142: - Summary: Update metrics documentation for errors and new metrics Key: KAFKA-16142 URL: https://issues.apache.org/jira/browse/KAFKA-16142 Project: Kafka Issue

[PR] Kafka 16131: Only update directoryIds if the metadata version supports DirectoryAssignment [kafka]

2024-01-15 Thread via GitHub
pprovenzano opened a new pull request, #15197: URL: https://github.com/apache/kafka/pull/15197 We only want to send directory assignments if the metadata version supports it. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to

Re: [PR] [Kafka-14404] fix overlap of streams-config sections & describe additional parameters [kafka]

2024-01-15 Thread via GitHub
ableegoldman commented on code in PR #15162: URL: https://github.com/apache/kafka/pull/15162#discussion_r1452785627 ## docs/streams/developer-guide/config-streams.html: ## @@ -1097,8 +1097,8 @@ Naming Default Values Kafka Streams uses different default values for

[jira] [Updated] (KAFKA-16032) Review client errors thrown on OffsetFetch and OffsetCommit failures

2024-01-15 Thread Lianet Magrans (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16032?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Lianet Magrans updated KAFKA-16032: --- Description: OffsetFetch and OffsetCommit handle errors separately. There are some issues

[jira] [Commented] (KAFKA-14507) Add ConsumerGroupPrepareAssignment API

2024-01-15 Thread Phuc Hong Tran (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-14507?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17807006#comment-17807006 ] Phuc Hong Tran commented on KAFKA-14507: sure [~linzihao1999]  > Add

[jira] [Commented] (KAFKA-14508) Add ConsumerGroupInstallAssignment API

2024-01-15 Thread Phuc Hong Tran (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-14508?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17807005#comment-17807005 ] Phuc Hong Tran commented on KAFKA-14508: thanks [~alexanderaghili]  > Add

[jira] [Assigned] (KAFKA-14507) Add ConsumerGroupPrepareAssignment API

2024-01-15 Thread Phuc Hong Tran (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-14507?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Phuc Hong Tran reassigned KAFKA-14507: -- Assignee: Phuc Hong Tran (was: Zihao Lin) > Add ConsumerGroupPrepareAssignment API

[jira] [Assigned] (KAFKA-14508) Add ConsumerGroupInstallAssignment API

2024-01-15 Thread Phuc Hong Tran (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-14508?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Phuc Hong Tran reassigned KAFKA-14508: -- Assignee: Phuc Hong Tran > Add ConsumerGroupInstallAssignment API >

[jira] [Updated] (KAFKA-16135) kafka.api.PlaintextConsumerTest.testPerPartitionLeadMetricsCleanUpWithSubscribe(String, String).quorum=kraft+kip848.groupProtocol=consumer is flaky

2024-01-15 Thread Kirk True (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16135?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Kirk True updated KAFKA-16135: -- Fix Version/s: 3.8.0 >

[jira] [Updated] (KAFKA-16134) kafka.api.PlaintextConsumerTest.testPerPartitionLagMetricsCleanUpWithSubscribe(String, String).quorum=kraft+kip848.groupProtocol=consumer is flaky

2024-01-15 Thread Kirk True (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16134?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Kirk True updated KAFKA-16134: -- Component/s: clients consumer >

[jira] [Updated] (KAFKA-16134) kafka.api.PlaintextConsumerTest.testPerPartitionLagMetricsCleanUpWithSubscribe(String, String).quorum=kraft+kip848.groupProtocol=consumer is flaky

2024-01-15 Thread Kirk True (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16134?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Kirk True updated KAFKA-16134: -- Labels: consumer-threading-refactor kip-848-client-support (was: ) >

[jira] [Updated] (KAFKA-16029) Investigate cause of "Unable to find FetchSessionHandler for node X" in logs

2024-01-15 Thread Kirk True (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16029?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Kirk True updated KAFKA-16029: -- Fix Version/s: 3.8.0 > Investigate cause of "Unable to find FetchSessionHandler for node X" in logs >

[jira] [Created] (KAFKA-16143) New metrics for KIP-848 protocol

2024-01-15 Thread Kirk True (Jira)
Kirk True created KAFKA-16143: - Summary: New metrics for KIP-848 protocol Key: KAFKA-16143 URL: https://issues.apache.org/jira/browse/KAFKA-16143 Project: Kafka Issue Type: Task

[jira] [Resolved] (KAFKA-15809) Update broker's metadata schema to include TS enable status

2024-01-15 Thread Phuc Hong Tran (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15809?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Phuc Hong Tran resolved KAFKA-15809. Resolution: Won't Fix > Update broker's metadata schema to include TS enable status >

[jira] [Assigned] (KAFKA-15341) Enabling TS for a topic during rolling restart causes problems

2024-01-15 Thread Phuc Hong Tran (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15341?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Phuc Hong Tran reassigned KAFKA-15341: -- Assignee: (was: Phuc Hong Tran) > Enabling TS for a topic during rolling restart

[jira] [Assigned] (KAFKA-15682) Ensure internal remote log metadata topic does not expire its segments before deleting user-topic segments

2024-01-15 Thread Phuc Hong Tran (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15682?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Phuc Hong Tran reassigned KAFKA-15682: -- Assignee: (was: Phuc Hong Tran) > Ensure internal remote log metadata topic does

Re: [PR] KAFKA-16133: Reconciliation auto-commit fix [kafka]

2024-01-15 Thread via GitHub
lucasbru closed pull request #15195: KAFKA-16133: Reconciliation auto-commit fix URL: https://github.com/apache/kafka/pull/15195 -- 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.

Re: [PR] KAFKA-15853: Move KafkaConfig.Defaults to server module [kafka]

2024-01-15 Thread via GitHub
OmniaGM commented on PR #15158: URL: https://github.com/apache/kafka/pull/15158#issuecomment-1892798596 > @OmniaGM Can we rebase to resolve the conflicts? Thanks done now. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to

[jira] [Created] (KAFKA-16144) Controller leader checkQuorum timer should skip only 1 controller case

2024-01-15 Thread Luke Chen (Jira)
Luke Chen created KAFKA-16144: - Summary: Controller leader checkQuorum timer should skip only 1 controller case Key: KAFKA-16144 URL: https://issues.apache.org/jira/browse/KAFKA-16144 Project: Kafka

[jira] [Assigned] (KAFKA-16131) Repeated UnsupportedVersionException logged when running Kafka 3.7.0-RC2 KRaft cluster with metadata version 3.6

2024-01-15 Thread Proven Provenzano (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16131?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Proven Provenzano reassigned KAFKA-16131: - Assignee: Proven Provenzano > Repeated UnsupportedVersionException logged when

Re: [PR] KAFKA-16101: KRaft migration documentation is incorrect [kafka]

2024-01-15 Thread via GitHub
cmccabe commented on code in PR #15193: URL: https://github.com/apache/kafka/pull/15193#discussion_r1452787635 ## docs/ops.html: ## @@ -3956,25 +3956,31 @@ Reverting to ZooKeeper mode During the Migration While the cluster is still in migration mode, it is possible to

Re: [PR] [Kafka-14404] fix overlap of streams-config sections & describe additional parameters [kafka]

2024-01-15 Thread via GitHub
ableegoldman commented on code in PR #15162: URL: https://github.com/apache/kafka/pull/15162#discussion_r1452787287 ## docs/streams/developer-guide/config-streams.html: ## @@ -,28 +,31 @@ Default ValuesConsumer earliest -linger.ms +

Re: [PR] KAFKA-16101: KRaft migration documentation is incorrect [kafka]

2024-01-15 Thread via GitHub
cmccabe commented on PR #15193: URL: https://github.com/apache/kafka/pull/15193#issuecomment-1892848417 You are not supposed to remove `controller.quorum.voters` or `controller.listener.names` -- This is an automated message from the Apache Git Service. To respond to the message, please

Re: [PR] KAFKA-16133 - Reconciliation auto-commit fix [kafka]

2024-01-15 Thread via GitHub
lucasbru merged PR #15194: URL: https://github.com/apache/kafka/pull/15194 -- 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-16029: Fix "Unable to find FetchSessionHandler for node X" bug [kafka]

2024-01-15 Thread via GitHub
kirktrue commented on PR #15186: URL: https://github.com/apache/kafka/pull/15186#issuecomment-1892622425 Test failures from most recent build are known flaky tests. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the

[jira] [Updated] (KAFKA-15935) Flaky test: testRestartReplication() – org.apache.kafka.connect.mirror.integration.IdentityReplicationIntegrationTest

2024-01-15 Thread Kirk True (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15935?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Kirk True updated KAFKA-15935: -- Labels: flaky (was: ) > Flaky test: testRestartReplication() – >

Re: [PR] KAFKA-16029: Fix "Unable to find FetchSessionHandler for node X" bug [kafka]

2024-01-15 Thread via GitHub
philipnee commented on code in PR #15186: URL: https://github.com/apache/kafka/pull/15186#discussion_r1452662028 ## clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractFetch.java: ## @@ -376,25 +376,21 @@ protected Map prepareCloseFetchSessi

[jira] [Updated] (KAFKA-15935) Flaky test: testRestartReplication() – org.apache.kafka.connect.mirror.integration.IdentityReplicationIntegrationTest

2024-01-15 Thread Kirk True (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15935?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Kirk True updated KAFKA-15935: -- Labels: flaky-test (was: flaky) > Flaky test: testRestartReplication() – >

Re: [PR] KAFKA-16029: Fix "Unable to find FetchSessionHandler for node X" bug [kafka]

2024-01-15 Thread via GitHub
kirktrue commented on code in PR #15186: URL: https://github.com/apache/kafka/pull/15186#discussion_r1452664088 ## clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractFetch.java: ## @@ -376,25 +376,21 @@ protected Map prepareCloseFetchSessi final

Re: [PR] KAFKA-16029: Fix "Unable to find FetchSessionHandler for node X" bug [kafka]

2024-01-15 Thread via GitHub
kirktrue commented on code in PR #15186: URL: https://github.com/apache/kafka/pull/15186#discussion_r1452665083 ## clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractFetch.java: ## @@ -376,25 +376,21 @@ protected Map prepareCloseFetchSessi final

[jira] [Updated] (KAFKA-16029) Investigate cause of "Unable to find FetchSessionHandler for node X" in logs

2024-01-15 Thread Kirk True (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16029?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Kirk True updated KAFKA-16029: -- Fix Version/s: 3.7.0 (was: 3.8.0) > Investigate cause of "Unable to find

Re: [PR] MINOR: Uniformize error handling/transformation in GroupCoordinatorService [kafka]

2024-01-15 Thread via GitHub
dajac commented on PR #15196: URL: https://github.com/apache/kafka/pull/15196#issuecomment-1892630714 Need to fix checkstyle. -- 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

[jira] [Updated] (KAFKA-16133) Commits during reconciliation always time out

2024-01-15 Thread Kirk True (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16133?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Kirk True updated KAFKA-16133: -- Fix Version/s: 3.8.0 > Commits during reconciliation always time out >

[jira] [Updated] (KAFKA-16032) Review client errors on OffsetFetch and OffsetCommit

2024-01-15 Thread Lianet Magrans (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16032?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Lianet Magrans updated KAFKA-16032: --- Summary: Review client errors on OffsetFetch and OffsetCommit (was: Review client error

[jira] [Updated] (KAFKA-16032) Review client errors on OffsetFetch and OffsetCommit

2024-01-15 Thread Lianet Magrans (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16032?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Lianet Magrans updated KAFKA-16032: --- Description: OffsetFetch and OffsetCommit handle errors separately. There are 2 issues to

[jira] [Updated] (KAFKA-16032) Review client errors thrown on OffsetFetch and OffsetCommit failures

2024-01-15 Thread Lianet Magrans (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16032?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Lianet Magrans updated KAFKA-16032: --- Summary: Review client errors thrown on OffsetFetch and OffsetCommit failures (was: Review

Re: [PR] KAFKA-15853: Move ClientQuotaManagerConfig outside of core [kafka]

2024-01-15 Thread via GitHub
OmniaGM commented on PR #15159: URL: https://github.com/apache/kafka/pull/15159#issuecomment-1892768226 > Hello @OmniaGM > > Can you, please, omit force push to the PR's? :) This breaks commit history and reviewers (like I am) don't understand what have changed with force push. >

Re: [PR] KAFKA-15853: Move ClientQuotaManagerConfig outside of core [kafka]

2024-01-15 Thread via GitHub
OmniaGM commented on PR #15159: URL: https://github.com/apache/kafka/pull/15159#issuecomment-1892768464 > @OmniaGM This needs rebasing too. Thanks done -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL

[jira] [Commented] (KAFKA-16120) Fix partition reassignment during ZK migration

2024-01-15 Thread Ismael Juma (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16120?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17807041#comment-17807041 ] Ismael Juma commented on KAFKA-16120: - [~cmccabe] If you're working on KAFKA-14616, assign the issue

[jira] [Commented] (KAFKA-14616) Topic recreation with offline broker causes permanent URPs

2024-01-15 Thread Ismael Juma (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-14616?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17807042#comment-17807042 ] Ismael Juma commented on KAFKA-14616: - [~cmccabe] Is this a blocker for 3.7.0? > Topic recreation

[jira] [Updated] (KAFKA-16132) Upgrading from 3.6 to 3.7 in KRaft will have seconds of partitions unavailable

2024-01-15 Thread Ismael Juma (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16132?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-16132: Priority: Blocker (was: Major) > Upgrading from 3.6 to 3.7 in KRaft will have seconds of

[jira] [Commented] (KAFKA-16132) Upgrading from 3.6 to 3.7 in KRaft will have seconds of partitions unavailable

2024-01-15 Thread Ismael Juma (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16132?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17807046#comment-17807046 ] Ismael Juma commented on KAFKA-16132: - OK, marked it as a blocker for now. > Upgrading from 3.6 to

[jira] [Updated] (KAFKA-16118) Coordinator unloading fails when replica is deleted

2024-01-15 Thread David Jacot (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16118?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] David Jacot updated KAFKA-16118: Affects Version/s: 3.6.0 (was: 3.7.0) > Coordinator unloading fails

[jira] [Commented] (KAFKA-16121) Partition reassignments in ZK migration dual write mode stalled until leader epoch incremented

2024-01-15 Thread Ismael Juma (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16121?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17807040#comment-17807040 ] Ismael Juma commented on KAFKA-16121: - It's a bit odd to mark this as a duplicate of KAFKA-16120

[jira] [Commented] (KAFKA-16132) Upgrading from 3.6 to 3.7 in KRaft will have seconds of partitions unavailable

2024-01-15 Thread Colin McCabe (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16132?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17807044#comment-17807044 ] Colin McCabe commented on KAFKA-16132: -- I think we need to look at this more, but it may be a

[jira] [Updated] (KAFKA-16118) Coordinator unloading fails when replica is deleted

2024-01-15 Thread David Jacot (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16118?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] David Jacot updated KAFKA-16118: Fix Version/s: 3.7.0 (was: 3.8.0) > Coordinator unloading fails when

Re: [PR] KAFKA-14505; [6/N] Avoid recheduling callback in request thread [kafka]

2024-01-15 Thread via GitHub
dajac commented on code in PR #15176: URL: https://github.com/apache/kafka/pull/15176#discussion_r1453042161 ## core/src/main/scala/kafka/server/ReplicaManager.scala: ## @@ -1090,38 +1090,29 @@ class ReplicaManager(val config: KafkaConfig, * @param producerId the

[jira] [Updated] (KAFKA-16120) Fix partition reassignment during ZK migration

2024-01-15 Thread Colin McCabe (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16120?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Colin McCabe updated KAFKA-16120: - Description: When a reassignment is completed in ZK migration hybrid mode, the `StopReplica`

[jira] [Resolved] (KAFKA-16121) Partition reassignments in ZK migration dual write mode stalled until leader epoch incremented

2024-01-15 Thread Colin McCabe (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16121?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Colin McCabe resolved KAFKA-16121. -- Fix Version/s: 3.7.0 Reviewer: Colin McCabe Assignee: David Mao

[jira] [Updated] (KAFKA-16120) Fix partition reassignment during ZK migration

2024-01-15 Thread Colin McCabe (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16120?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Colin McCabe updated KAFKA-16120: - Summary: Fix partition reassignment during ZK migration (was: Partition reassignments in ZK

[jira] [Commented] (KAFKA-16132) Upgrading from 3.6 to 3.7 in KRaft will have seconds of partitions unavailable

2024-01-15 Thread Ismael Juma (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16132?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17807043#comment-17807043 ] Ismael Juma commented on KAFKA-16132: - [~cmccabe] Is this a blocker for 3.7.0? > Upgrading from 3.6

[jira] [Assigned] (KAFKA-15724) KRaft support in OffsetFetchRequestTest

2024-01-15 Thread Shivsundar R (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15724?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Shivsundar R reassigned KAFKA-15724: Assignee: Shivsundar R > KRaft support in OffsetFetchRequestTest >

Re: [PR] KAFKA-15181: Improvements for TopicBaseRemoteLogMetadataManager [kafka]

2024-01-15 Thread via GitHub
showuon commented on code in PR #14127: URL: https://github.com/apache/kafka/pull/14127#discussion_r1450149078 ## storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/ConsumerTask.java: ## @@ -64,302 +63,403 @@ class ConsumerTask implements Runnable,

[jira] [Assigned] (KAFKA-15727) KRaft support in AlterUserScramCredentialsRequestNotAuthorizedTest

2024-01-15 Thread Abhinav Dixit (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15727?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Abhinav Dixit reassigned KAFKA-15727: - Assignee: Abhinav Dixit > KRaft support in

Re: [PR] KAFKA-15181: Improvements for TopicBaseRemoteLogMetadataManager [kafka]

2024-01-15 Thread via GitHub
kamalcph commented on code in PR #14127: URL: https://github.com/apache/kafka/pull/14127#discussion_r1452997033 ## storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/ConsumerTask.java: ## @@ -64,302 +63,403 @@ class ConsumerTask implements Runnable,

Re: [PR] KAFKA-14505; [6/N] Avoid recheduling callback in request thread [kafka]

2024-01-15 Thread via GitHub
dajac commented on code in PR #15176: URL: https://github.com/apache/kafka/pull/15176#discussion_r1453039755 ## core/src/main/scala/kafka/server/ReplicaManager.scala: ## @@ -1132,36 +1123,31 @@ class ReplicaManager(val config: KafkaConfig, * @param transactionalId

[PR] KAFKA-16133 - Reconciliation auto-commit fix [kafka]

2024-01-15 Thread via GitHub
lianetm opened a new pull request, #15194: URL: https://github.com/apache/kafka/pull/15194 This fixes an issue with the time boundaries used for the auto-commit performed when partitions are revoked. -- This is an automated message from the Apache Git Service. To respond to the

[PR] KAFKA-16133: Reconciliation auto-commit fix [kafka]

2024-01-15 Thread via GitHub
lucasbru opened a new pull request, #15195: URL: https://github.com/apache/kafka/pull/15195 This fixes an issue with the time boundaries used for the auto-commit performed when partitions are revoked. ### Committer Checklist (excluded from commit message) - [ ] Verify design and

[jira] [Created] (KAFKA-16136) CoordinatorTest.testTaskRequestWithOldStartMsGetsUpdated() is very flaky

2024-01-15 Thread Stanislav Kozlovski (Jira)
Stanislav Kozlovski created KAFKA-16136: --- Summary: CoordinatorTest.testTaskRequestWithOldStartMsGetsUpdated() is very flaky Key: KAFKA-16136 URL: https://issues.apache.org/jira/browse/KAFKA-16136

Re: [PR] KAFKA-16133 - Reconciliation auto-commit fix [kafka]

2024-01-15 Thread via GitHub
lucasbru commented on code in PR #15194: URL: https://github.com/apache/kafka/pull/15194#discussion_r1452516993 ## clients/src/main/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImpl.java: ## @@ -854,6 +854,14 @@ boolean reconcile() { return true;

Re: [PR] KAFKA-16133 - Reconciliation auto-commit fix [kafka]

2024-01-15 Thread via GitHub
lucasbru commented on PR #15194: URL: https://github.com/apache/kafka/pull/15194#issuecomment-1892460560 LGTM, thanks! -- 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

[jira] [Commented] (KAFKA-16134) kafka.api.PlaintextConsumerTest.testPerPartitionLagMetricsCleanUpWithSubscribe(String, String).quorum=kraft+kip848.groupProtocol=consumer is flaky

2024-01-15 Thread Stanislav Kozlovski (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16134?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17806872#comment-17806872 ] Stanislav Kozlovski commented on KAFKA-16134: - Locally it passed after a few tries.

  1   2   >