[jira] [Commented] (KAFKA-16282) Allow to get last stable offset (LSO) in kafka-get-offsets.sh

2024-02-26 Thread Luke Chen (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16282?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17820943#comment-17820943 ] Luke Chen commented on KAFKA-16282: --- [~ahmedsobeh] , thanks for the drafted KIP. I think you can refer

Re: [PR] KAFKA-15215: migrate StreamedJoinTest to Mockito [kafka]

2024-02-26 Thread via GitHub
ableegoldman merged PR #15424: URL: https://github.com/apache/kafka/pull/15424 -- 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-15215: migrate StreamedJoinTest to Mockito [kafka]

2024-02-26 Thread via GitHub
ableegoldman commented on PR #15424: URL: https://github.com/apache/kafka/pull/15424#issuecomment-1965693809 Looks like a new build was triggered when I renamed the PR but I have the results from before that and it looked good. Test failures all unrelated. Will merge this to trunk

Re: [PR] KAFKA-15215: migrate StreamedJoinTest to Mockito [kafka]

2024-02-26 Thread via GitHub
ableegoldman commented on PR #15424: URL: https://github.com/apache/kafka/pull/15424#issuecomment-1965688725 remember to name your PRs correctly -- I added the `KAFKA-15215` prefix since this came about from that ticket, but you can always use `MINOR:` for small fixes that don't have a

Re: [PR] KAFKA-15215: migrate StreamedJoinTest to Mockito [kafka]

2024-02-26 Thread via GitHub
agavra commented on code in PR #15424: URL: https://github.com/apache/kafka/pull/15424#discussion_r1503563459 ## build.gradle: ## @@ -2333,6 +2333,8 @@ project(':streams:streams-scala') { testImplementation libs.junitJupiter testImplementation libs.easymock +

Re: [PR] KAFKA-15215: migrate StreamedJoinTest to Mockito [kafka]

2024-02-26 Thread via GitHub
ableegoldman commented on code in PR #15424: URL: https://github.com/apache/kafka/pull/15424#discussion_r1503562979 ## build.gradle: ## @@ -2333,6 +2333,8 @@ project(':streams:streams-scala') { testImplementation libs.junitJupiter testImplementation libs.easymock +

[jira] [Updated] (KAFKA-16102) about DynamicListenerConfig, the dynamic modification of the listener's port or IP does not take effect.

2024-02-26 Thread Jialun Peng (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16102?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jialun Peng updated KAFKA-16102: Description: When I dynamically modify the parameters related to Kafka listeners, such as

Re: [PR] KAFKA-16154: Broker returns offset for LATEST_TIERED_TIMESTAMP [kafka]

2024-02-26 Thread via GitHub
showuon commented on PR #15213: URL: https://github.com/apache/kafka/pull/15213#issuecomment-1965678417 @clolov , do we have any update about the compilation error fix? -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use

Re: [PR] KAFKA-16226 Add test for concurrently updatingMetadata and fetching snapshot/cluster [kafka]

2024-02-26 Thread via GitHub
hachikuji merged PR #15385: URL: https://github.com/apache/kafka/pull/15385 -- 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-16305: Avoid optimisation in handshakeUnwrap [kafka]

2024-02-26 Thread via GitHub
gaurav-narula commented on code in PR #15434: URL: https://github.com/apache/kafka/pull/15434#discussion_r1503432226 ## clients/src/test/java/org/apache/kafka/common/security/ssl/NettySslEngineFactory.java: ## @@ -0,0 +1,159 @@ +package org.apache.kafka.common.security.ssl; +

Re: [PR] KAFKA-16294: Add group protocol migration enabling config [kafka]

2024-02-26 Thread via GitHub
jeffkbkim commented on code in PR #15411: URL: https://github.com/apache/kafka/pull/15411#discussion_r1503404075 ## group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupProtocolMigrationConfig.java: ## @@ -0,0 +1,68 @@ +/* + * Licensed to the Apache Software

Re: [PR] MINOR: Fix UpdatedImage and HighWatermarkUpdated events' logs [kafka]

2024-02-26 Thread via GitHub
jolshan commented on code in PR #15432: URL: https://github.com/apache/kafka/pull/15432#discussion_r1503408163 ## group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/CoordinatorRuntime.java: ## @@ -1731,15 +1735,19 @@ public void onNewMetadataImage(

[PR] MINOR: Add 3.7.0 to system tests [kafka]

2024-02-26 Thread via GitHub
stanislavkozlovski opened a new pull request, #15436: URL: https://github.com/apache/kafka/pull/15436 As per the [release instructions](https://cwiki.apache.org/confluence/display/KAFKA/Release+Process), bumping the versions in the associated files here as part of the 3.7 release --

Re: [PR] MINOR: Optimize EventAccumulator [kafka]

2024-02-26 Thread via GitHub
jolshan commented on code in PR #15430: URL: https://github.com/apache/kafka/pull/15430#discussion_r1503400099 ## group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/EventAccumulator.java: ## @@ -137,31 +136,43 @@ public void add(T event) throws

Re: [PR] MINOR: Optimize EventAccumulator [kafka]

2024-02-26 Thread via GitHub
jeffkbkim commented on code in PR #15430: URL: https://github.com/apache/kafka/pull/15430#discussion_r1503398332 ## group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/EventAccumulator.java: ## @@ -137,31 +136,43 @@ public void add(T event) throws

Re: [PR] MINOR: Optimize EventAccumulator [kafka]

2024-02-26 Thread via GitHub
jeffkbkim commented on code in PR #15430: URL: https://github.com/apache/kafka/pull/15430#discussion_r1503396742 ## group-coordinator/src/test/java/org/apache/kafka/coordinator/group/runtime/MultiThreadedEventProcessorTest.java: ## @@ -53,53 +50,19 @@ @Timeout(value = 60)

[jira] [Commented] (KAFKA-16277) CooperativeStickyAssignor does not spread topics evenly among consumer group

2024-02-26 Thread A. Sophie Blee-Goldman (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16277?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17820880#comment-17820880 ] A. Sophie Blee-Goldman commented on KAFKA-16277: FYI I added you as a contributor so you

[jira] [Resolved] (KAFKA-16277) CooperativeStickyAssignor does not spread topics evenly among consumer group

2024-02-26 Thread A. Sophie Blee-Goldman (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16277?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] A. Sophie Blee-Goldman resolved KAFKA-16277. Resolution: Fixed > CooperativeStickyAssignor does not spread topics

[jira] [Updated] (KAFKA-16277) CooperativeStickyAssignor does not spread topics evenly among consumer group

2024-02-26 Thread A. Sophie Blee-Goldman (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16277?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] A. Sophie Blee-Goldman updated KAFKA-16277: --- Fix Version/s: 3.8.0 > CooperativeStickyAssignor does not spread topics

[jira] [Assigned] (KAFKA-16277) CooperativeStickyAssignor does not spread topics evenly among consumer group

2024-02-26 Thread A. Sophie Blee-Goldman (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16277?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] A. Sophie Blee-Goldman reassigned KAFKA-16277: -- Assignee: Cameron Redpath > CooperativeStickyAssignor does not

Re: [PR] KAFKA-16277: AbstractStickyAssignor - Sort owned TopicPartitions by partition when reassigning [kafka]

2024-02-26 Thread via GitHub
ableegoldman commented on PR #15416: URL: https://github.com/apache/kafka/pull/15416#issuecomment-1965331000 Test failures are unrelated, merged to trunk. Thanks for the fix! -- This is an automated message from the Apache Git Service. To respond to the message, please log on to

Re: [PR] KAFKA-16277: AbstractStickyAssignor - Sort owned TopicPartitions by partition when reassigning [kafka]

2024-02-26 Thread via GitHub
ableegoldman merged PR #15416: URL: https://github.com/apache/kafka/pull/15416 -- 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-16277: AbstractStickyAssignor - Sort owned TopicPartitions by partition when reassigning [kafka]

2024-02-26 Thread via GitHub
ableegoldman commented on code in PR #15416: URL: https://github.com/apache/kafka/pull/15416#discussion_r1503326278 ## clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractStickyAssignor.java: ## @@ -663,6 +663,7 @@ private void assignOwnedPartitions() {

[PR] MINOR: remove test constructor for PartitionAssignment [kafka]

2024-02-26 Thread via GitHub
cmccabe opened a new pull request, #15435: URL: https://github.com/apache/kafka/pull/15435 Remove the test constructor for PartitionAssignment and remove the TODO. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the

[jira] [Created] (KAFKA-16308) Formatting and Updating Kafka Features

2024-02-26 Thread Justine Olshan (Jira)
Justine Olshan created KAFKA-16308: -- Summary: Formatting and Updating Kafka Features Key: KAFKA-16308 URL: https://issues.apache.org/jira/browse/KAFKA-16308 Project: Kafka Issue Type: Task

Re: [PR] MINOR: Optimize EventAccumulator [kafka]

2024-02-26 Thread via GitHub
jolshan commented on PR #15430: URL: https://github.com/apache/kafka/pull/15430#issuecomment-1965265194 Left one question -- otherwise lgtm -- 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

Re: [PR] MINOR: Optimize EventAccumulator [kafka]

2024-02-26 Thread via GitHub
jolshan commented on code in PR #15430: URL: https://github.com/apache/kafka/pull/15430#discussion_r1503291909 ## group-coordinator/src/test/java/org/apache/kafka/coordinator/group/runtime/MultiThreadedEventProcessorTest.java: ## @@ -53,53 +50,19 @@ @Timeout(value = 60)

Re: [PR] KAFKA-15585: DescribeTopicPartitions client side change [kafka]

2024-02-26 Thread via GitHub
kirktrue commented on code in PR #15265: URL: https://github.com/apache/kafka/pull/15265#discussion_r1503266038 ## clients/src/main/java/org/apache/kafka/common/TopicPartitionInfo.java: ## @@ -79,9 +95,24 @@ public List isr() { return isr; } +/** + *

[jira] [Commented] (KAFKA-15695) Local log start offset is not updated on the follower after rebuilding remote log auxiliary state

2024-02-26 Thread Henry Cai (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15695?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17820852#comment-17820852 ] Henry Cai commented on KAFKA-15695: --- Can this fix be back ported into 3.6.X branch? > Local log start

Re: [PR] KAFKA-15585: DescribeTopicPartitions client side change [kafka]

2024-02-26 Thread via GitHub
kirktrue commented on code in PR #15265: URL: https://github.com/apache/kafka/pull/15265#discussion_r1503175975 ## clients/src/main/java/org/apache/kafka/clients/admin/DescribeTopicsOptions.java: ## @@ -47,8 +49,25 @@ public DescribeTopicsOptions

[jira] [Commented] (KAFKA-16282) Allow to get last stable offset (LSO) in kafka-get-offsets.sh

2024-02-26 Thread Justine Olshan (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16282?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17820821#comment-17820821 ] Justine Olshan commented on KAFKA-16282: Thanks [~ahmedsobeh]  A few things: For the Admin.java

Re: [PR] KAFKA-16305: Avoid optimisation in handshakeUnwrap [kafka]

2024-02-26 Thread via GitHub
ijuma commented on code in PR #15434: URL: https://github.com/apache/kafka/pull/15434#discussion_r1503149170 ## clients/src/test/java/org/apache/kafka/common/security/ssl/NettySslEngineFactory.java: ## @@ -0,0 +1,159 @@ +package org.apache.kafka.common.security.ssl; + +import

[jira] [Commented] (KAFKA-9062) Handle stalled writes to RocksDB

2024-02-26 Thread Matthias J. Sax (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-9062?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17820819#comment-17820819 ] Matthias J. Sax commented on KAFKA-9062: Given that bulk loading was disabled, should we close

[jira] [Assigned] (KAFKA-14919) MM2 ForwardingAdmin tests should not conflate admin operations

2024-02-26 Thread Anton Liauchuk (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-14919?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Anton Liauchuk reassigned KAFKA-14919: -- Assignee: Anton Liauchuk > MM2 ForwardingAdmin tests should not conflate admin

[jira] [Commented] (KAFKA-16223) Replace EasyMock and PowerMock with Mockito for KafkaConfigBackingStoreTest

2024-02-26 Thread Chaitanya Mukka (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16223?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17820801#comment-17820801 ] Chaitanya Mukka commented on KAFKA-16223: - Ah sure. Let me do that. Currently, the tests have

Re: [PR] MINOR: Optimize EventAccumulator [kafka]

2024-02-26 Thread via GitHub
jolshan commented on code in PR #15430: URL: https://github.com/apache/kafka/pull/15430#discussion_r1503073175 ## group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/EventAccumulator.java: ## @@ -137,31 +136,43 @@ public void add(T event) throws

Re: [PR] migrate StreamedJoinTest to Mockito [kafka]

2024-02-26 Thread via GitHub
agavra commented on code in PR #15424: URL: https://github.com/apache/kafka/pull/15424#discussion_r1503025048 ## streams/streams-scala/src/test/scala/org/apache/kafka/streams/scala/kstream/StreamJoinedTest.scala: ## @@ -21,24 +21,21 @@ import

[jira] [Assigned] (KAFKA-16307) fix EventAccumulator thread idle ratio metric

2024-02-26 Thread David Jacot (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16307?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] David Jacot reassigned KAFKA-16307: --- Assignee: Jeff Kim > fix EventAccumulator thread idle ratio metric >

[jira] [Updated] (KAFKA-16307) fix EventAccumulator thread idle ratio metric

2024-02-26 Thread Jeff Kim (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16307?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jeff Kim updated KAFKA-16307: - Description: The metric does not seem to be accurate, nor reporting metrics at every interval. Requires

[jira] [Updated] (KAFKA-16307) fix EventAccumulator thread idle ratio metric

2024-02-26 Thread Jeff Kim (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16307?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jeff Kim updated KAFKA-16307: - Summary: fix EventAccumulator thread idle ratio metric (was: investigate EventAccumulator thread idle

[jira] [Updated] (KAFKA-16307) investigate EventAccumulator thread idle ratio metric

2024-02-26 Thread Jeff Kim (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16307?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jeff Kim updated KAFKA-16307: - Description: The metric does not seem to  > investigate EventAccumulator thread idle ratio metric >

[jira] [Created] (KAFKA-16307) investigate EventAccumulator thread idle ratio metric

2024-02-26 Thread Jeff Kim (Jira)
Jeff Kim created KAFKA-16307: Summary: investigate EventAccumulator thread idle ratio metric Key: KAFKA-16307 URL: https://issues.apache.org/jira/browse/KAFKA-16307 Project: Kafka Issue Type:

[jira] [Updated] (KAFKA-16307) fix EventAccumulator thread idle ratio metric

2024-02-26 Thread Jeff Kim (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16307?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jeff Kim updated KAFKA-16307: - Description: The metric does not seem to be accurate. Requires investigation (was: The metric does not

[jira] [Assigned] (KAFKA-16306) GroupCoordinatorService logger is not configured

2024-02-26 Thread Anton Liauchuk (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16306?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Anton Liauchuk reassigned KAFKA-16306: -- Assignee: Jeff Kim (was: Anton Liauchuk) > GroupCoordinatorService logger is not

[PR] KAFKA-16306: fix GroupCoordinatorService logger [kafka]

2024-02-26 Thread via GitHub
jeffkbkim opened a new pull request, #15433: URL: https://github.com/apache/kafka/pull/15433 *More detailed description of your change, if necessary. The PR title and PR message become the squashed commit message, so use a separate comment to ping reviewers.* *Summary of

[jira] [Assigned] (KAFKA-16306) GroupCoordinatorService logger is not configured

2024-02-26 Thread David Jacot (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16306?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] David Jacot reassigned KAFKA-16306: --- Assignee: Jeff Kim > GroupCoordinatorService logger is not configured >

[jira] [Assigned] (KAFKA-16306) GroupCoordinatorService logger is not configured

2024-02-26 Thread Anton Liauchuk (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16306?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Anton Liauchuk reassigned KAFKA-16306: -- Assignee: Anton Liauchuk (was: Jeff Kim) > GroupCoordinatorService logger is not

[jira] [Created] (KAFKA-16306) GroupCoordinatorService logger is not configured

2024-02-26 Thread Jeff Kim (Jira)
Jeff Kim created KAFKA-16306: Summary: GroupCoordinatorService logger is not configured Key: KAFKA-16306 URL: https://issues.apache.org/jira/browse/KAFKA-16306 Project: Kafka Issue Type:

[jira] [Commented] (KAFKA-16160) AsyncKafkaConsumer is trying to connect to a disconnected node in a tight loop

2024-02-26 Thread Kirk True (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16160?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17820747#comment-17820747 ] Kirk True commented on KAFKA-16160: --- [~pnee]—do you have some insights on how [~phuctran] can

[jira] [Commented] (KAFKA-16190) Member should send full heartbeat when rejoining

2024-02-26 Thread Kirk True (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16190?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17820746#comment-17820746 ] Kirk True commented on KAFKA-16190: --- [~lucasbru]—could you take a look at the pull request for this

[PR] MINOR: Fix UpdatedImage and HighWatermarkUpdated events' logs [kafka]

2024-02-26 Thread via GitHub
dajac opened a new pull request, #15432: URL: https://github.com/apache/kafka/pull/15432 I have noticed the following log when a __consumer_offsets partition immigrate from a broker. It appends because the event is queued up after the event that unloads the state machine. This patch fixes

Re: [PR] KAFKA-16249; Improve reconciliation state machine [kafka]

2024-02-26 Thread via GitHub
dajac commented on PR #15364: URL: https://github.com/apache/kafka/pull/15364#issuecomment-1964189888 @jeffkbkim Thanks for your comments. I have addressed all of them. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use

Re: [PR] KAFKA-16249; Improve reconciliation state machine [kafka]

2024-02-26 Thread via GitHub
dajac commented on code in PR #15364: URL: https://github.com/apache/kafka/pull/15364#discussion_r1502633032 ## group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java: ## @@ -2079,253 +2056,20 @@ public void testReconciliationProcess()

[jira] [Comment Edited] (KAFKA-16282) Allow to get last stable offset (LSO) in kafka-get-offsets.sh

2024-02-26 Thread Ahmed Sobeh (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16282?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17820677#comment-17820677 ] Ahmed Sobeh edited comment on KAFKA-16282 at 2/26/24 1:23 PM: -- Thanks both

[jira] [Comment Edited] (KAFKA-16282) Allow to get last stable offset (LSO) in kafka-get-offsets.sh

2024-02-26 Thread Ahmed Sobeh (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16282?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17820677#comment-17820677 ] Ahmed Sobeh edited comment on KAFKA-16282 at 2/26/24 1:23 PM: -- Thanks both

Re: [PR] migrate StreamedJoinTest to Mockito [kafka]

2024-02-26 Thread via GitHub
divijvaidya commented on code in PR #15424: URL: https://github.com/apache/kafka/pull/15424#discussion_r1502590907 ## streams/streams-scala/src/test/scala/org/apache/kafka/streams/scala/kstream/StreamJoinedTest.scala: ## @@ -21,24 +21,21 @@ import

[jira] [Updated] (KAFKA-16305) Optimisation in SslTransportLayer:handshakeUnwrap stalls TLS handshake

2024-02-26 Thread Gaurav Narula (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16305?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gaurav Narula updated KAFKA-16305: -- Summary: Optimisation in SslTransportLayer:handshakeUnwrap stalls TLS handshake (was: #

[jira] [Created] (KAFKA-16305) # Optimisation in SslTransportLayer:handshakeUnwrap stalls TLS handshake

2024-02-26 Thread Gaurav Narula (Jira)
Gaurav Narula created KAFKA-16305: - Summary: # Optimisation in SslTransportLayer:handshakeUnwrap stalls TLS handshake Key: KAFKA-16305 URL: https://issues.apache.org/jira/browse/KAFKA-16305 Project:

[jira] [Assigned] (KAFKA-16305) # Optimisation in SslTransportLayer:handshakeUnwrap stalls TLS handshake

2024-02-26 Thread Gaurav Narula (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16305?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gaurav Narula reassigned KAFKA-16305: - Assignee: Gaurav Narula > # Optimisation in SslTransportLayer:handshakeUnwrap stalls

Re: [PR] KAFKA-16226 Add test for concurrently updatingMetadata and fetching snapshot/cluster [kafka]

2024-02-26 Thread via GitHub
msn-tldr commented on code in PR #15385: URL: https://github.com/apache/kafka/pull/15385#discussion_r1502561721 ## clients/src/test/java/org/apache/kafka/clients/MetadataTest.java: ## @@ -1260,6 +1265,103 @@ else if (partition.equals(internalPart))

Re: [PR] KAFKA-16226 Add test for concurrently updatingMetadata and fetching snapshot/cluster [kafka]

2024-02-26 Thread via GitHub
msn-tldr commented on code in PR #15385: URL: https://github.com/apache/kafka/pull/15385#discussion_r1502561721 ## clients/src/test/java/org/apache/kafka/clients/MetadataTest.java: ## @@ -1260,6 +1265,103 @@ else if (partition.equals(internalPart))

[jira] [Commented] (KAFKA-16282) Allow to get last stable offset (LSO) in kafka-get-offsets.sh

2024-02-26 Thread Ahmed Sobeh (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16282?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17820677#comment-17820677 ] Ahmed Sobeh commented on KAFKA-16282: - Thanks both! I just finished writing up the

[PR] MINOR: Bump 3.7 branch's version to 3.7.1-SNAPSHOT [kafka]

2024-02-26 Thread via GitHub
stanislavkozlovski opened a new pull request, #15431: URL: https://github.com/apache/kafka/pull/15431 This patch updates the 3.7 release branch's version to 3.7.1-SNAPSHOT as per the wiki process -- This is an automated message from the Apache Git Service. To respond to the message,

Re: [PR] KAFKA-15716: KRaft support in EpochDrivenReplicationProtocolAcceptanceTest [kafka]

2024-02-26 Thread via GitHub
mimaison commented on code in PR #15295: URL: https://github.com/apache/kafka/pull/15295#discussion_r1502397230 ## core/src/test/scala/unit/kafka/server/epoch/EpochDrivenReplicationProtocolAcceptanceTest.scala: ## @@ -53,17 +58,26 @@ class

[jira] [Commented] (KAFKA-16008) Fix PlaintextConsumerTest.testMaxPollIntervalMs

2024-02-26 Thread Lucas Brutschy (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16008?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17820663#comment-17820663 ] Lucas Brutschy commented on KAFKA-16008: Fixed by

[jira] [Resolved] (KAFKA-16008) Fix PlaintextConsumerTest.testMaxPollIntervalMs

2024-02-26 Thread Lucas Brutschy (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16008?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Lucas Brutschy resolved KAFKA-16008. Resolution: Duplicate > Fix PlaintextConsumerTest.testMaxPollIntervalMs >

Re: [PR] KAFKA-16258: callback to release assignment when stale member leaves group [kafka]

2024-02-26 Thread via GitHub
lucasbru merged PR #15415: URL: https://github.com/apache/kafka/pull/15415 -- 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:

[PR] MINOR: Optimize EventAccumulator [kafka]

2024-02-26 Thread via GitHub
dajac opened a new pull request, #15430: URL: https://github.com/apache/kafka/pull/15430 `poll(long timeout, TimeUnit unit)` is either used with `Long.MAX_VALUE` or `0`. This patch replaces it with `poll` and `take`. It removes the `awaitNanos` usage. ### Committer Checklist

Re: [PR] KAFKA-16066: Upgrade apacheds to 2.0.0.AM27 With apache kerby [kafka]

2024-02-26 Thread via GitHub
mimaison commented on code in PR #15277: URL: https://github.com/apache/kafka/pull/15277#discussion_r1502366164 ## core/src/test/scala/kafka/security/minikdc/MiniKdc.scala: ## @@ -19,38 +19,22 @@ package kafka.security.minikdc import java.io._ -import

[jira] [Updated] (KAFKA-16105) Reassignment of tiered topics is failing due to RemoteStorageException

2024-02-26 Thread Anatolii Popov (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16105?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Anatolii Popov updated KAFKA-16105: --- Labels: tiered-storage (was: ) > Reassignment of tiered topics is failing due to

Re: [PR] KAFKA-14747: record discarded FK join subscription responses [kafka]

2024-02-26 Thread via GitHub
AyoubOm commented on PR #15395: URL: https://github.com/apache/kafka/pull/15395#issuecomment-1963731421 @mjsax Please have a look when you have some time -- 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] MINOR: Fix ReadOnlySessionStore java docs [kafka]

2024-02-26 Thread via GitHub
AyoubOm commented on PR #15412: URL: https://github.com/apache/kafka/pull/15412#issuecomment-1963723331 @mimaison Please check this when you have some time -- 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-15462: Add Group Type Filter for List Group to the Admin Client [kafka]

2024-02-26 Thread via GitHub
dajac commented on code in PR #15150: URL: https://github.com/apache/kafka/pull/15150#discussion_r1502279590 ## tools/src/test/java/org/apache/kafka/tools/consumer/group/ListConsumerGroupTest.java: ## @@ -20,83 +20,258 @@ import kafka.admin.ConsumerGroupCommand; import