Re: [PR] KAFKA-16745: Implemented handleShareFetchRequest RPC including unit tests [kafka]

2024-07-11 Thread via GitHub
chirag-wadhwa5 commented on code in PR #16456: URL: https://github.com/apache/kafka/pull/16456#discussion_r1675361989 ## core/src/main/scala/kafka/server/KafkaApis.scala: ## @@ -3955,11 +3948,484 @@ class KafkaApis(val requestChannel: RequestChannel, } } + /** + * H

[jira] [Commented] (KAFKA-17124) Fix flaky DumpLogSegmentsTest#testDumpRemoteLogMetadataNonZeroStartingOffset

2024-07-11 Thread Federico Valeri (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-17124?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17865316#comment-17865316 ] Federico Valeri commented on KAFKA-17124: - I mean, I thought I fixed that flakin

[jira] [Commented] (KAFKA-17124) Fix flaky DumpLogSegmentsTest#testDumpRemoteLogMetadataNonZeroStartingOffset

2024-07-11 Thread Chia-Ping Tsai (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-17124?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17865313#comment-17865313 ] Chia-Ping Tsai commented on KAFKA-17124: Could you please share the commit to me

[jira] [Commented] (KAFKA-17124) Fix flaky DumpLogSegmentsTest#testDumpRemoteLogMetadataNonZeroStartingOffset

2024-07-11 Thread Federico Valeri (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-17124?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17865311#comment-17865311 ] Federico Valeri commented on KAFKA-17124: - [~chia7712] I thought I fixed that. A

Re: [PR] KAFKA-17061 Improve the performance of isReplicaOnline [kafka]

2024-07-11 Thread via GitHub
chia7712 commented on PR #16529: URL: https://github.com/apache/kafka/pull/16529#issuecomment-2224616092 @ocadaruma thanks for this improvement! -- 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 th

Re: [PR] KAFKA-17061 Improve the performance of isReplicaOnline [kafka]

2024-07-11 Thread via GitHub
chia7712 merged PR #16529: URL: https://github.com/apache/kafka/pull/16529 -- 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: jira-unsubscr...@kafka.apache

[jira] [Resolved] (KAFKA-17061) KafkaController takes long time to connect to newly added broker after registration on large cluster

2024-07-11 Thread Chia-Ping Tsai (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-17061?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Chia-Ping Tsai resolved KAFKA-17061. Fix Version/s: 3.9.0 Resolution: Fixed > KafkaController takes long time to connect

[jira] [Created] (KAFKA-17124) Fix flaky DumpLogSegmentsTest#testDumpRemoteLogMetadataNonZeroStartingOffset

2024-07-11 Thread Chia-Ping Tsai (Jira)
Chia-Ping Tsai created KAFKA-17124: -- Summary: Fix flaky DumpLogSegmentsTest#testDumpRemoteLogMetadataNonZeroStartingOffset Key: KAFKA-17124 URL: https://issues.apache.org/jira/browse/KAFKA-17124 Proj

Re: [PR] KAFKA-16791: Add thread detection to ClusterTestExtensions [kafka]

2024-07-11 Thread via GitHub
chia7712 commented on code in PR #16499: URL: https://github.com/apache/kafka/pull/16499#discussion_r1675284548 ## core/src/test/java/kafka/test/junit/ClusterTestExtensions.java: ## @@ -119,7 +124,27 @@ public Stream provideTestTemplateInvocationContex return generated

Re: [PR] KAFKA-17102: FetchRequest#forgottenTopics would return incorrect data [kafka]

2024-07-11 Thread via GitHub
m1a2st commented on PR #16557: URL: https://github.com/apache/kafka/pull/16557#issuecomment-2224534471 @chia7712, Thanks for reminder, rebased it -- 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 t

Re: [PR] MINOR:Topic command integration test migrate to new test infra [kafka]

2024-07-11 Thread via GitHub
chia7712 commented on code in PR #16127: URL: https://github.com/apache/kafka/pull/16127#discussion_r1675282098 ## clients/src/test/java/org/apache/kafka/test/TestUtils.java: ## @@ -706,4 +706,9 @@ public static ApiVersionsResponse createApiVersionsResponse( Api

Re: [PR] KAFKA-17102: FetchRequest#forgottenTopics would return incorrect data [kafka]

2024-07-11 Thread via GitHub
chia7712 commented on PR #16557: URL: https://github.com/apache/kafka/pull/16557#issuecomment-2224529187 @m1a2st please rebase code to include your recent fix to trigger QA -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and u

Re: [PR] KAFKA-17096:Fix kafka_log4j_appender.py [kafka]

2024-07-11 Thread via GitHub
chia7712 commented on PR #16559: URL: https://github.com/apache/kafka/pull/16559#issuecomment-2224474099 @gongxuanzhang Could you test this patch on your local and then attach the result? -- This is an automated message from the Apache Git Service. To respond to the message, please log on

Re: [PR] MINOR: add clean up command to tests README [kafka]

2024-07-11 Thread via GitHub
chia7712 commented on code in PR #16560: URL: https://github.com/apache/kafka/pull/16560#discussion_r1675083256 ## tests/README.md: ## @@ -47,6 +47,10 @@ TC_PATHS="tests/kafkatest/tests/streams/streams_upgrade_test.py::StreamsUpgradeT ``` bash tests/docker/ducker-ak up -j 'op

Re: [PR] KAFKA-17055: Change KafkaRaftClientTest and KafkaRaftClientSnapshotTest nodes to use random positive number id [kafka]

2024-07-11 Thread via GitHub
masonyc commented on code in PR #16563: URL: https://github.com/apache/kafka/pull/16563#discussion_r1675023567 ## raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java: ## @@ -281,15 +281,20 @@ public void testGrantVotesWhenShuttingDown(boolean withKip853Rpc) throws

Re: [PR] KAFKA-17055: Change KafkaRaftClientTest and KafkaRaftClientSnapshotTest nodes to use random positive number id [kafka]

2024-07-11 Thread via GitHub
masonyc commented on code in PR #16563: URL: https://github.com/apache/kafka/pull/16563#discussion_r1675024064 ## raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientSnapshotTest.java: ## @@ -1977,12 +1978,20 @@ private static ReplicaKey replicaKey(int id, boolean withDirec

Re: [PR] KAFKA-17055: Change KafkaRaftClientTest and KafkaRaftClientSnapshotTest nodes to use random positive number id [kafka]

2024-07-11 Thread via GitHub
masonyc commented on code in PR #16563: URL: https://github.com/apache/kafka/pull/16563#discussion_r1675023791 ## raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java: ## @@ -3860,12 +3865,11 @@ private static ReplicaKey replicaKey(int id, boolean withDirectoryId)

Re: [PR] MINOR: add clean up command to tests README [kafka]

2024-07-11 Thread via GitHub
gongxuanzhang commented on PR #16560: URL: https://github.com/apache/kafka/pull/16560#issuecomment-2224264080 @chia7712 thanks for your review,I update it -- 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

Re: [PR] KAFKA-17096:Fix kafka_log4j_appender.py [kafka]

2024-07-11 Thread via GitHub
gongxuanzhang commented on PR #16559: URL: https://github.com/apache/kafka/pull/16559#issuecomment-2224263676 @chia7712 thanks for your review,I update it -- 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

[jira] [Resolved] (KAFKA-10356) Handle accidental deletion of sink-topics as exceptional failure

2024-07-11 Thread Matthias J. Sax (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-10356?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax resolved KAFKA-10356. - Resolution: Not A Problem > Handle accidental deletion of sink-topics as exceptional fai

[jira] [Commented] (KAFKA-10356) Handle accidental deletion of sink-topics as exceptional failure

2024-07-11 Thread Matthias J. Sax (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-10356?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17865263#comment-17865263 ] Matthias J. Sax commented on KAFKA-10356: - Talked to a few people, and there was

Re: [PR] KAFKA-15773: Group protocol configuration should be validated [kafka]

2024-07-11 Thread via GitHub
kirktrue commented on code in PR #16543: URL: https://github.com/apache/kafka/pull/16543#discussion_r1674890318 ## clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java: ## @@ -720,6 +721,12 @@ private void maybeOverrideEnableAutoCommit(Map configs) {

[PR] KAFKA-16765: NioEchoServer leaks accepted SocketChannel instances due to race condition [kafka]

2024-07-11 Thread via GitHub
zzzk1 opened a new pull request, #16576: URL: https://github.com/apache/kafka/pull/16576 As described from [KAFKA-16765](https://issues.apache.org/jira/browse/KAFKA-16765): There may be channel in `newChannels` that are not closed in time (NioEchoServer#run can handle it, but the process

[jira] [Commented] (KAFKA-17097) Add replace.null.with.default configuration to ValueToKey and ReplaceField (KIP-1040)

2024-07-11 Thread PoAn Yang (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-17097?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17865251#comment-17865251 ] PoAn Yang commented on KAFKA-17097: --- PR: https://github.com/apache/kafka/pull/16571 >

Re: [PR] KAFKA-17110: Enable valid test case in KafkaConsumerTest for AsyncKafkaConsumer [kafka]

2024-07-11 Thread via GitHub
FrankYang0529 commented on PR #16566: URL: https://github.com/apache/kafka/pull/16566#issuecomment-2224098005 > @FrankYang0529 please rebase code to trigger KafkaConsuemrTest Rebased it. Thank you. -- This is an automated message from the Apache Git Service. To respond to the messag

Re: [PR] KAFKA-17077: The node.id is inconsistent to broker.id when "broker.id.generation.enable=true". [kafka]

2024-07-11 Thread via GitHub
chia7712 commented on code in PR #16540: URL: https://github.com/apache/kafka/pull/16540#discussion_r1674724394 ## core/src/main/scala/kafka/server/KafkaConfig.scala: ## @@ -308,7 +308,9 @@ class KafkaConfig private(doLog: Boolean, val props: util.Map[_, _]) val brokerIdGene

Re: [PR] KAFKA-17104: fix InvalidMessageCrcRecordsPerSec is not updated in validating Legac… [kafka]

2024-07-11 Thread via GitHub
chia7712 commented on code in PR #16558: URL: https://github.com/apache/kafka/pull/16558#discussion_r1674721981 ## core/src/test/scala/unit/kafka/log/LogValidatorTest.scala: ## @@ -687,6 +687,57 @@ class LogValidatorTest { verifyRecordValidationStats(validatedResults.record

Re: [PR] KAFKA-17096:Fix kafka_log4j_appender.py [kafka]

2024-07-11 Thread via GitHub
chia7712 commented on code in PR #16559: URL: https://github.com/apache/kafka/pull/16559#discussion_r1674720250 ## build.gradle: ## @@ -1211,6 +1211,8 @@ project(':core') { //By default gradle does not handle test dependencies between the sub-projects //This line is t

Re: [PR] MINOR: add clean up command to tests README [kafka]

2024-07-11 Thread via GitHub
chia7712 commented on code in PR #16560: URL: https://github.com/apache/kafka/pull/16560#discussion_r1674716809 ## tests/README.md: ## @@ -47,6 +47,10 @@ TC_PATHS="tests/kafkatest/tests/streams/streams_upgrade_test.py::StreamsUpgradeT ``` bash tests/docker/ducker-ak up -j 'op

Re: [PR] KAFKA-17110: Enable valid test case in KafkaConsumerTest for AsyncKafkaConsumer [kafka]

2024-07-11 Thread via GitHub
chia7712 commented on PR #16566: URL: https://github.com/apache/kafka/pull/16566#issuecomment-2223956814 @FrankYang0529 please rebase code to trigger KafkaConsuemrTest -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use t

Re: [PR] KAFKA-17106:enable testFetchProgressWithMissingPartitionPosition [kafka]

2024-07-11 Thread via GitHub
chia7712 commented on PR #16564: URL: https://github.com/apache/kafka/pull/16564#issuecomment-2223956208 @TaiJuWu could you please rebase code to check the hangings ? -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the

[jira] [Resolved] (KAFKA-17092) Revisit `KafkaConsumerTest#testBeginningOffsetsTimeout` for AsyncConsumer

2024-07-11 Thread Chia-Ping Tsai (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-17092?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Chia-Ping Tsai resolved KAFKA-17092. Fix Version/s: 3.9.0 Resolution: Fixed > Revisit `KafkaConsumerTest#testBeginningOf

Re: [PR] KAFKA-17092: Revisit `KafkaConsumerTest#testBeginningOffsetsTimeout` for AsyncConsumer [kafka]

2024-07-11 Thread via GitHub
chia7712 merged PR #16541: URL: https://github.com/apache/kafka/pull/16541 -- 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: jira-unsubscr...@kafka.apache

Re: [PR] KAFKA-17061 Improve the performance of isReplicaOnline [kafka]

2024-07-11 Thread via GitHub
chia7712 commented on code in PR #16529: URL: https://github.com/apache/kafka/pull/16529#discussion_r1674687739 ## jmh-benchmarks/src/main/java/org/apache/kafka/jmh/controller/ControllerContextBenchmark.java: ## @@ -0,0 +1,74 @@ +/* + * Licensed to the Apache Software Foundation

Re: [PR] KAFKA-17115: Send LeaveGroup request if consumer receives JoinGroup response with member ID after being closed [kafka]

2024-07-11 Thread via GitHub
C0urante commented on PR #16567: URL: https://github.com/apache/kafka/pull/16567#issuecomment-2223928868 @showuon would you have a moment to take a look at this one? I believe it's been the culprit for most of the failures in the Connect `OffsetsApiIntegrationTest` suite and I'm hoping to i

Re: [PR] MINOR: Rewrite the meta.properties handling code in Java and fix some issues [kafka]

2024-07-11 Thread via GitHub
chia7712 commented on code in PR #14628: URL: https://github.com/apache/kafka/pull/14628#discussion_r1674672583 ## core/src/main/scala/kafka/tools/StorageTool.scala: ## @@ -60,7 +63,11 @@ object StorageTool extends Logging { if (!metadataVersion.isKRaftSupported) {

[jira] [Commented] (KAFKA-16855) KRaft - Wire replaying a TopicRecord

2024-07-11 Thread Muralidhar Basani (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16855?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17865217#comment-17865217 ] Muralidhar Basani commented on KAFKA-16855: --- [~christo_lolov] can you pls prov

[jira] [Commented] (KAFKA-17098) Error Opening RocksDBStore

2024-07-11 Thread Eduwer Camacaro (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-17098?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17865210#comment-17865210 ] Eduwer Camacaro commented on KAFKA-17098: - Yes it works! thanks! > Error Openin

Re: [PR] MINOR: Rewrite the meta.properties handling code in Java and fix some issues [kafka]

2024-07-11 Thread via GitHub
ijuma commented on code in PR #14628: URL: https://github.com/apache/kafka/pull/14628#discussion_r1674625790 ## core/src/main/scala/kafka/tools/StorageTool.scala: ## @@ -60,7 +63,11 @@ object StorageTool extends Logging { if (!metadataVersion.isKRaftSupported) {

[jira] [Assigned] (KAFKA-17105) Unnecessary connector restarts after being newly created

2024-07-11 Thread Chris Egerton (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-17105?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Chris Egerton reassigned KAFKA-17105: - Assignee: Chris Egerton > Unnecessary connector restarts after being newly created > --

[jira] [Resolved] (KAFKA-14710) KRaft cannot initialise storage for valid UUID

2024-07-11 Thread Chia-Ping Tsai (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-14710?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Chia-Ping Tsai resolved KAFKA-14710. Resolution: Won't Fix > KRaft cannot initialise storage for valid UUID > -

[jira] [Commented] (KAFKA-14710) KRaft cannot initialise storage for valid UUID

2024-07-11 Thread Chia-Ping Tsai (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-14710?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17865204#comment-17865204 ] Chia-Ping Tsai commented on KAFKA-14710: The uuid check was remove by https://gi

[jira] [Assigned] (KAFKA-17118) Remove StorageTool#buildMetadataProperties

2024-07-11 Thread Chia-Ping Tsai (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-17118?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Chia-Ping Tsai reassigned KAFKA-17118: -- Assignee: kangning.li (was: Chia-Ping Tsai) > Remove StorageTool#buildMetadataProper

Re: [PR] MINOR: Add logs when metadata update is not successful [kafka]

2024-07-11 Thread via GitHub
subhashiyer9 commented on code in PR #16496: URL: https://github.com/apache/kafka/pull/16496#discussion_r1674584267 ## clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java: ## @@ -1201,13 +1201,26 @@ private ClusterAndWaitTime waitOnMetadata(String topic,

Re: [PR] MINOR: Rewrite the meta.properties handling code in Java and fix some issues [kafka]

2024-07-11 Thread via GitHub
chia7712 commented on code in PR #14628: URL: https://github.com/apache/kafka/pull/14628#discussion_r1674554451 ## core/src/main/scala/kafka/tools/StorageTool.scala: ## @@ -60,7 +63,11 @@ object StorageTool extends Logging { if (!metadataVersion.isKRaftSupported) {

[jira] [Created] (KAFKA-17123) update docs of Storage Tool to clarify that "cluster id" can be non-uuid format after 3.7

2024-07-11 Thread Chia-Ping Tsai (Jira)
Chia-Ping Tsai created KAFKA-17123: -- Summary: update docs of Storage Tool to clarify that "cluster id" can be non-uuid format after 3.7 Key: KAFKA-17123 URL: https://issues.apache.org/jira/browse/KAFKA-17123

[jira] [Created] (KAFKA-17122) Change the type of `clusterId` from `UUID` to `String`

2024-07-11 Thread Chia-Ping Tsai (Jira)
Chia-Ping Tsai created KAFKA-17122: -- Summary: Change the type of `clusterId` from `UUID` to `String` Key: KAFKA-17122 URL: https://issues.apache.org/jira/browse/KAFKA-17122 Project: Kafka Is

Re: [PR] MINOR: Rewrite the meta.properties handling code in Java and fix some issues [kafka]

2024-07-11 Thread via GitHub
chia7712 commented on code in PR #14628: URL: https://github.com/apache/kafka/pull/14628#discussion_r1674554451 ## core/src/main/scala/kafka/tools/StorageTool.scala: ## @@ -60,7 +63,11 @@ object StorageTool extends Logging { if (!metadataVersion.isKRaftSupported) {

[jira] [Commented] (KAFKA-16221) IllegalStateException from Producer

2024-07-11 Thread sabeywic (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16221?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17865193#comment-17865193 ] sabeywic commented on KAFKA-16221: -- Im still facing this issue in version 3.7. Do you g

[jira] [Commented] (KAFKA-16855) KRaft - Wire replaying a TopicRecord

2024-07-11 Thread Muralidhar Basani (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16855?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17865191#comment-17865191 ] Muralidhar Basani commented on KAFKA-16855: --- [~showuon] sure. Will start looki

[PR] MINOR: Remove incompatible system ducktape upgrade tests [kafka]

2024-07-11 Thread via GitHub
lzyLuke opened a new pull request, #16575: URL: https://github.com/apache/kafka/pull/16575 ![image](https://github.com/apache/kafka/assets/27408588/852fd2ec-fea4-4a64-8bcb-5976e09d4dfb) We are seeing ducktape tests failure on upgrade tests. The thing is some tests should not w

Re: [PR] KAFKA-17111: Revert "KAFKA-15996: Improve JsonConverter performance (#14992)" [kafka]

2024-07-11 Thread via GitHub
gharris1727 closed pull request #16568: KAFKA-17111: Revert "KAFKA-15996: Improve JsonConverter performance (#14992)" URL: https://github.com/apache/kafka/pull/16568 -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL

[PR] KAFKA-16743,KAFKA-16744: KafkaApis support for share group heartbeat and describe (KIP-932) [kafka]

2024-07-11 Thread via GitHub
apoorvmittal10 opened a new pull request, #16574: URL: https://github.com/apache/kafka/pull/16574 Added handling of share group heartbeat and describe in KafkaApis. The Implementation of heartbeat and describe is with group coordinator. ### Committer Checklist (excluded from commit me

Re: [PR] MINOR: Fix transactions_upgrade_test to run transactions during upgrade [kafka]

2024-07-11 Thread via GitHub
jolshan merged PR #16462: URL: https://github.com/apache/kafka/pull/16462 -- 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: jira-unsubscr...@kafka.apache.

Re: [PR] KAFKA-16772: Introduce kraft.version to support KIP-853 [kafka]

2024-07-11 Thread via GitHub
jsancio commented on code in PR #16230: URL: https://github.com/apache/kafka/pull/16230#discussion_r1674397738 ## clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsResponse.java: ## @@ -313,10 +313,12 @@ private static FinalizedFeatureKeyCollection createFinaliz

Re: [PR] MINOR: Fix transactions_upgrade_test to run transactions during upgrade [kafka]

2024-07-11 Thread via GitHub
jolshan commented on PR #16462: URL: https://github.com/apache/kafka/pull/16462#issuecomment-2223500451 Finally got to confirming the test results 👍 All passed. will merge now. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub a

Re: [PR] KAFKA-17055: Change KafkaRaftClientTest and KafkaRaftClientSnapshotTest nodes to use random positive number id [kafka]

2024-07-11 Thread via GitHub
jsancio commented on code in PR #16563: URL: https://github.com/apache/kafka/pull/16563#discussion_r1674382183 ## raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java: ## @@ -281,15 +281,20 @@ public void testGrantVotesWhenShuttingDown(boolean withKip853Rpc) throws

[jira] [Commented] (KAFKA-16937) Consider inlineing Time#waitObject to ProducerMetadata#awaitUpdate

2024-07-11 Thread Greg Harris (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16937?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17865164#comment-17865164 ] Greg Harris commented on KAFKA-16937: - I thought about this some more. Right now the

[jira] [Comment Edited] (KAFKA-16937) Consider inlineing Time#waitObject to ProducerMetadata#awaitUpdate

2024-07-11 Thread Greg Harris (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16937?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17865164#comment-17865164 ] Greg Harris edited comment on KAFKA-16937 at 7/11/24 5:15 PM:

Re: [PR] KAFKA-15999 Migrate HeartbeatRequestManagerTest away from ConsumerTestBuilder [kafka]

2024-07-11 Thread via GitHub
lianetm commented on code in PR #16200: URL: https://github.com/apache/kafka/pull/16200#discussion_r1674318250 ## clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java: ## @@ -447,7 +505,8 @@ private ConsumerGroupHeartbeatRequest get

[jira] [Updated] (KAFKA-9738) Add Generics Type Parameters to forwarded() in MockProcessorContext

2024-07-11 Thread Matthias J. Sax (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-9738?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax updated KAFKA-9738: --- Description: The method {{forwarded()}} to capture the forwarded records in {{MockProcessorC

[jira] [Updated] (KAFKA-9738) Add Generics Type Parameters to forwarded() in MockProcessorContext

2024-07-11 Thread Matthias J. Sax (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-9738?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax updated KAFKA-9738: --- Labels: kip (was: ) > Add Generics Type Parameters to forwarded() in MockProcessorContext >

Re: [PR] MINOR: Add logs when metadata update is not successful [kafka]

2024-07-11 Thread via GitHub
jolshan commented on code in PR #16496: URL: https://github.com/apache/kafka/pull/16496#discussion_r1674314203 ## clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java: ## @@ -1201,13 +1201,26 @@ private ClusterAndWaitTime waitOnMetadata(String topic, Integ

Re: [PR] KAFKA-16192: Introduce transaction.version and usage of flexible records to coordinators [kafka]

2024-07-11 Thread via GitHub
jolshan commented on code in PR #16183: URL: https://github.com/apache/kafka/pull/16183#discussion_r1674310171 ## server-common/src/main/java/org/apache/kafka/server/common/MetadataVersion.java: ## @@ -217,8 +217,11 @@ public enum MetadataVersion { // Add ELR related suppor

Re: [PR] KAFKA-15999 Migrate HeartbeatRequestManagerTest away from ConsumerTestBuilder [kafka]

2024-07-11 Thread via GitHub
lianetm commented on code in PR #16200: URL: https://github.com/apache/kafka/pull/16200#discussion_r1674309494 ## clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java: ## @@ -425,6 +482,7 @@ topicId, mkSortedSet(0) assertEqu

[jira] [Assigned] (KAFKA-9738) Add Generics Type Parameters to forwarded() in MockProcessorContext

2024-07-11 Thread Matthias J. Sax (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-9738?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax reassigned KAFKA-9738: -- Assignee: Matthias J. Sax > Add Generics Type Parameters to forwarded() in MockProcess

[jira] [Commented] (KAFKA-9738) Add Generics Type Parameters to forwarded() in MockProcessorContext

2024-07-11 Thread Matthias J. Sax (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-9738?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17865145#comment-17865145 ] Matthias J. Sax commented on KAFKA-9738: Given that `Processor` is deprecated alr

[jira] [Resolved] (KAFKA-10814) improving ability of handling exception in kafka

2024-07-11 Thread Matthias J. Sax (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-10814?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax resolved KAFKA-10814. - Resolution: Incomplete Unclear what the problem is. Closing. > improving ability of han

Re: [PR] KAFKA-16192: Introduce transaction.version and usage of flexible records to coordinators [kafka]

2024-07-11 Thread via GitHub
junrao commented on code in PR #16183: URL: https://github.com/apache/kafka/pull/16183#discussion_r1674294082 ## server-common/src/main/java/org/apache/kafka/server/common/MetadataVersion.java: ## @@ -217,8 +217,11 @@ public enum MetadataVersion { // Add ELR related support

[PR] KAFKA-16741: Add ShareGroupHeartbeat API support - 2/N (KIP-932) [kafka]

2024-07-11 Thread via GitHub
apoorvmittal10 opened a new pull request, #16573: URL: https://github.com/apache/kafka/pull/16573 ShareGroupHeartbeat API support as defined in KIP-932. The heartbeat persists Group and Member information on `__consumer_offsets` topic. The PR also moves some of the ShareGroupConfigs t

[PR] KAFKA-17073: Deprecate ReplicaVerificationTool in 3.9 [kafka]

2024-07-11 Thread via GitHub
dongjinleekr opened a new pull request, #16572: URL: https://github.com/apache/kafka/pull/16572 ### Summary If the user runs `ReplicaVerificationTool`, it shows a deprecation warning. ### Committer Checklist (excluded from commit message) - [ ] Verify design and implementatio

Re: [PR] KAFKA-17098: Re-add task to state updater if transit to RUNNING fails [kafka]

2024-07-11 Thread via GitHub
jlprat commented on PR #16570: URL: https://github.com/apache/kafka/pull/16570#issuecomment-2223290582 Cherry picked to `trunk` -- 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 commen

[jira] [Commented] (KAFKA-17101) Mirror maker internal topics cleanup policy changes to 'delete' from 'compact'

2024-07-11 Thread Greg Harris (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-17101?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17865138#comment-17865138 ] Greg Harris commented on KAFKA-17101: - Also you mentioned MM1, is that a typo or are

[jira] [Commented] (KAFKA-17101) Mirror maker internal topics cleanup policy changes to 'delete' from 'compact'

2024-07-11 Thread Greg Harris (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-17101?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17865137#comment-17865137 ] Greg Harris commented on KAFKA-17101: - [~kaushik srinivas] Unfortunately I don't see

Re: [PR] KAFKA-17098: Re-add task to state updater if transit to RUNNING fails [kafka]

2024-07-11 Thread via GitHub
jlprat commented on PR #16570: URL: https://github.com/apache/kafka/pull/16570#issuecomment-2223277883 I'll cherry pick this one to `trunk` -- 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 spe

[jira] [Comment Edited] (KAFKA-17101) Mirror maker internal topics cleanup policy changes to 'delete' from 'compact'

2024-07-11 Thread Greg Harris (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-17101?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17864935#comment-17864935 ] Greg Harris edited comment on KAFKA-17101 at 7/11/24 3:40 PM:

Re: [PR] KAFKA-17098: Re-add task to state updater if transit to RUNNING fails [kafka]

2024-07-11 Thread via GitHub
mjsax commented on PR #16570: URL: https://github.com/apache/kafka/pull/16570#issuecomment-2223275885 Who is cherry-picking this to `trunk`? @jlprat are you doing this, or should we (@cadonna) do the cherry-pick? -- This is an automated message from the Apache Git Service. To respond to t

[jira] [Commented] (KAFKA-17117) Avoid instantiating classpath plugins when service loading plugins

2024-07-11 Thread Greg Harris (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-17117?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17865134#comment-17865134 ] Greg Harris commented on KAFKA-17117: - Considering that the only plugins typically o

[jira] [Updated] (KAFKA-17098) Error Opening RocksDBStore

2024-07-11 Thread Josep Prat (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-17098?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Josep Prat updated KAFKA-17098: --- Fix Version/s: 3.8.0 > Error Opening RocksDBStore > -- > > K

Re: [PR] KAFKA-17098: Re-add task to state updater if transit to RUNNING fails [kafka]

2024-07-11 Thread via GitHub
jlprat merged PR #16570: URL: https://github.com/apache/kafka/pull/16570 -- 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: jira-unsubscr...@kafka.apache.o

[jira] [Resolved] (KAFKA-17111) ServiceConfigurationError in JsonSerializer/Deserializer during Plugin Discovery

2024-07-11 Thread Greg Harris (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-17111?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Greg Harris resolved KAFKA-17111. - Resolution: Fixed > ServiceConfigurationError in JsonSerializer/Deserializer during Plugin > Di

[jira] [Resolved] (KAFKA-17119) After enabled kafka-ranger-plugin and baned the user for using describe in policy, but that user still can use describe.

2024-07-11 Thread Greg Harris (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-17119?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Greg Harris resolved KAFKA-17119. - Resolution: Cannot Reproduce > After enabled kafka-ranger-plugin and baned the user for using de

[jira] [Commented] (KAFKA-17119) After enabled kafka-ranger-plugin and baned the user for using describe in policy, but that user still can use describe.

2024-07-11 Thread Greg Harris (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-17119?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17865131#comment-17865131 ] Greg Harris commented on KAFKA-17119: - Hi [~StarBoy1005] The Apache Kafka project do

Re: [PR] KAFKA-15999 Migrate HeartbeatRequestManagerTest away from ConsumerTestBuilder [kafka]

2024-07-11 Thread via GitHub
lianetm commented on code in PR #16200: URL: https://github.com/apache/kafka/pull/16200#discussion_r1674210447 ## clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java: ## @@ -201,8 +202,42 @@ public void testSuccessfulHeartbeatTiming

Re: [PR] KAFKA-15999 Migrate HeartbeatRequestManagerTest away from ConsumerTestBuilder [kafka]

2024-07-11 Thread via GitHub
lianetm commented on code in PR #16200: URL: https://github.com/apache/kafka/pull/16200#discussion_r1674210447 ## clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java: ## @@ -201,8 +202,42 @@ public void testSuccessfulHeartbeatTiming

Re: [PR] KAFKA-15999 Migrate HeartbeatRequestManagerTest away from ConsumerTestBuilder [kafka]

2024-07-11 Thread via GitHub
lianetm commented on code in PR #16200: URL: https://github.com/apache/kafka/pull/16200#discussion_r1674207651 ## clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java: ## @@ -201,8 +202,42 @@ public void testSuccessfulHeartbeatTiming

Re: [PR] KAFKA-15999 Migrate HeartbeatRequestManagerTest away from ConsumerTestBuilder [kafka]

2024-07-11 Thread via GitHub
lianetm commented on code in PR #16200: URL: https://github.com/apache/kafka/pull/16200#discussion_r1674174436 ## clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java: ## @@ -659,95 +749,50 @@ public void testPollTimerExpirationShou

Re: [PR] KAFKA-15999 Migrate HeartbeatRequestManagerTest away from ConsumerTestBuilder [kafka]

2024-07-11 Thread via GitHub
lianetm commented on code in PR #16200: URL: https://github.com/apache/kafka/pull/16200#discussion_r1674165119 ## clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java: ## @@ -756,18 +801,19 @@ public void testFencedMemberStopHeartbe

Re: [PR] KAFKA-17061 Improve the performance of isReplicaOnline [kafka]

2024-07-11 Thread via GitHub
ocadaruma commented on PR #16529: URL: https://github.com/apache/kafka/pull/16529#issuecomment-2223158388 @chia7712 Added a flamegraph. PTAL https://issues.apache.org/jira/browse/KAFKA-17061?focusedCommentId=17865110&page=com.atlassian.jira.plugin.system.issuetabpanels%3Acomment-tabpanel#

[jira] [Commented] (KAFKA-17061) KafkaController takes long time to connect to newly added broker after registration on large cluster

2024-07-11 Thread Haruki Okada (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-17061?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17865110#comment-17865110 ] Haruki Okada commented on KAFKA-17061: -- Did a micro benchmark to check the performa

[jira] [Updated] (KAFKA-17061) KafkaController takes long time to connect to newly added broker after registration on large cluster

2024-07-11 Thread Haruki Okada (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-17061?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Haruki Okada updated KAFKA-17061: - Description: h2. Environment * Kafka version: 3.3.2 * Cluster: 200~ brokers * Total num parti

[jira] [Updated] (KAFKA-17061) KafkaController takes long time to connect to newly added broker after registration on large cluster

2024-07-11 Thread Haruki Okada (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-17061?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Haruki Okada updated KAFKA-17061: - Attachment: screenshot-flame-patched.png > KafkaController takes long time to connect to newly a

[jira] [Updated] (KAFKA-17061) KafkaController takes long time to connect to newly added broker after registration on large cluster

2024-07-11 Thread Haruki Okada (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-17061?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Haruki Okada updated KAFKA-17061: - Attachment: screenshot-flame.png > KafkaController takes long time to connect to newly added bro

Re: [PR] KAFKA-15999 Migrate HeartbeatRequestManagerTest away from ConsumerTestBuilder [kafka]

2024-07-11 Thread via GitHub
lianetm commented on code in PR #16200: URL: https://github.com/apache/kafka/pull/16200#discussion_r1674150218 ## clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java: ## @@ -251,23 +304,25 @@ public void testSkippingHeartbeat(final

[jira] [Updated] (KAFKA-17061) KafkaController takes long time to connect to newly added broker after registration on large cluster

2024-07-11 Thread Haruki Okada (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-17061?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Haruki Okada updated KAFKA-17061: - Attachment: flame-patched.html > KafkaController takes long time to connect to newly added broke

[jira] [Updated] (KAFKA-17061) KafkaController takes long time to connect to newly added broker after registration on large cluster

2024-07-11 Thread Haruki Okada (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-17061?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Haruki Okada updated KAFKA-17061: - Attachment: flame.html > KafkaController takes long time to connect to newly added broker after

[jira] [Updated] (KAFKA-17061) KafkaController takes long time to connect to newly added broker after registration on large cluster

2024-07-11 Thread Haruki Okada (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-17061?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Haruki Okada updated KAFKA-17061: - Description: h2. Environment * Kafka version: 3.3.2 * Cluster: 200~ brokers * Total num parti

Re: [PR] KAFKA-17017: AsyncKafkaConsumer#unsubscribe does not clean the assigned partitions [kafka]

2024-07-11 Thread via GitHub
FrankYang0529 commented on code in PR #16449: URL: https://github.com/apache/kafka/pull/16449#discussion_r1674145282 ## clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java: ## @@ -1501,11 +1501,11 @@ public void unsubscribe() { privat

Re: [PR] KAFKA-15999 Migrate HeartbeatRequestManagerTest away from ConsumerTestBuilder [kafka]

2024-07-11 Thread via GitHub
lianetm commented on code in PR #16200: URL: https://github.com/apache/kafka/pull/16200#discussion_r1674142299 ## clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java: ## @@ -201,8 +202,42 @@ public void testSuccessfulHeartbeatTiming

Re: [PR] KAFKA-15999 Migrate HeartbeatRequestManagerTest away from ConsumerTestBuilder [kafka]

2024-07-11 Thread via GitHub
lianetm commented on code in PR #16200: URL: https://github.com/apache/kafka/pull/16200#discussion_r1674138238 ## clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java: ## @@ -201,8 +202,42 @@ public void testSuccessfulHeartbeatTiming

  1   2   >