Re: [PR] KAFKA-16314: Introducing the AbortableTransactionException [kafka]

2024-03-18 Thread via GitHub
sjhajharia commented on code in PR #15486: URL: https://github.com/apache/kafka/pull/15486#discussion_r1529794514 ## core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala: ## @@ -3040,6 +3048,7 @@ class ReplicaManagerTest { transactionalId = transactionalId,

Re: [PR] KAFKA-16314: Introducing the AbortableTransactionException [kafka]

2024-03-18 Thread via GitHub
sjhajharia commented on code in PR #15486: URL: https://github.com/apache/kafka/pull/15486#discussion_r1529793846 ## clients/src/test/java/org/apache/kafka/clients/producer/internals/TransactionManagerTest.java: ## @@ -3523,6 +3524,128 @@ public void testForegroundInvalidStateT

Re: [PR] KAFKA-16341: fix the LogValidator for non-compressed type [kafka]

2024-03-18 Thread via GitHub
chia7712 commented on PR #15476: URL: https://github.com/apache/kafka/pull/15476#issuecomment-2005884703 I have rebase the PR to trigger QA again, and I will merge it if no related failed tests. -- This is an automated message from the Apache Git Service. To respond to the message, please

[jira] [Updated] (KAFKA-16222) KRaft Migration: Incorrect default user-principal quota after migration

2024-03-18 Thread Manikumar (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16222?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar updated KAFKA-16222: -- Fix Version/s: 3.6.2 3.8.0 3.7.1 > KRaft Migration: Incorrect de

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

2024-03-18 Thread Manikumar (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16073?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar updated KAFKA-16073: -- Fix Version/s: 3.7.1 > Kafka Tiered Storage: Consumer Fetch Error Due to Delayed localLogStartOffset

Re: [PR] KAFKA-16222: KRaft Migration: Incorrect default user-principal quota after migration [kafka]

2024-03-18 Thread via GitHub
omkreddy commented on PR #15481: URL: https://github.com/apache/kafka/pull/15481#issuecomment-2005776638 I am including this to 3.6.2 release plan -- 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

[jira] [Comment Edited] (KAFKA-13361) Support fine-grained compression options

2024-03-18 Thread Cheng-Kai, Zhang (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-13361?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17828150#comment-17828150 ] Cheng-Kai, Zhang edited comment on KAFKA-13361 at 3/19/24 4:05 AM: ---

[jira] [Created] (KAFKA-16384) KRaft controller number recommendation

2024-03-18 Thread Peter (Jira)
Peter created KAFKA-16384: - Summary: KRaft controller number recommendation Key: KAFKA-16384 URL: https://issues.apache.org/jira/browse/KAFKA-16384 Project: Kafka Issue Type: Bug Components

Re: [PR] KAFKA-14752: Kafka examples improvements - processor changes [kafka]

2024-03-18 Thread via GitHub
gaoran10 commented on code in PR #13516: URL: https://github.com/apache/kafka/pull/13516#discussion_r1529639335 ## examples/src/main/java/kafka/examples/ExactlyOnceMessageProcessor.java: ## @@ -81,111 +88,115 @@ public ExactlyOnceMessageProcessor(final String inputTopic,

Re: [PR] KAFKA-14752: Kafka examples improvements - processor changes [kafka]

2024-03-18 Thread via GitHub
gaoran10 commented on code in PR #13516: URL: https://github.com/apache/kafka/pull/13516#discussion_r1529639335 ## examples/src/main/java/kafka/examples/ExactlyOnceMessageProcessor.java: ## @@ -81,111 +88,115 @@ public ExactlyOnceMessageProcessor(final String inputTopic,

Re: [PR] KAFKA-16222: KRaft Migration: Incorrect default user-principal quota after migration [kafka]

2024-03-18 Thread via GitHub
showuon commented on PR #15481: URL: https://github.com/apache/kafka/pull/15481#issuecomment-2005672241 Re-triggering the CI build. -- 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 co

[jira] [Commented] (KAFKA-13361) Support fine-grained compression options

2024-03-18 Thread Cheng-Kai, Zhang (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-13361?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17828150#comment-17828150 ] Cheng-Kai, Zhang commented on KAFKA-13361: -- Hi [~dongjin]  [~mimaison] I am in

Re: [PR] KAFKA-16352: Txn may get get stuck in PrepareCommit or PrepareAbort s… [kafka]

2024-03-18 Thread via GitHub
jolshan merged PR #15524: URL: https://github.com/apache/kafka/pull/15524 -- 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-16352: Txn may get get stuck in PrepareCommit or PrepareAbort s… [kafka]

2024-03-18 Thread via GitHub
jolshan commented on PR #15524: URL: https://github.com/apache/kafka/pull/15524#issuecomment-2005610376 I am seeing it fail more often on your branch (for a few runs), but after merging with trunk, it seemed better. I will go ahead and merge. -- This is an automated message from the Apach

Re: [PR] KAFKA-16367; Full ConsumerGroupHeartbeat response must be sent when full request is received [kafka]

2024-03-18 Thread via GitHub
jolshan commented on code in PR #15533: URL: https://github.com/apache/kafka/pull/15533#discussion_r1529489216 ## group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java: ## @@ -1904,16 +2000,7 @@ public void testReconciliationProcess() {

Re: [PR] KAFKA-16367; Full ConsumerGroupHeartbeat response must be sent when full request is received [kafka]

2024-03-18 Thread via GitHub
jolshan commented on code in PR #15533: URL: https://github.com/apache/kafka/pull/15533#discussion_r1529486638 ## group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java: ## @@ -1190,10 +1190,11 @@ private CoordinatorResult consumerGr

Re: [PR] KAFKA-16367; Full ConsumerGroupHeartbeat response must be sent when full request is received [kafka]

2024-03-18 Thread via GitHub
jeffkbkim commented on PR #15533: URL: https://github.com/apache/kafka/pull/15533#issuecomment-2005338241 > This patch changes the logic to check ownedTopicPartitions, subscribedTopicNames and rebalanceTimeoutMs as they are the only three non optional fields. They are the only three

[jira] [Assigned] (KAFKA-14359) Idempotent Producer continues to retry on OutOfOrderSequence error when first batch fails

2024-03-18 Thread Justine Olshan (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-14359?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Justine Olshan reassigned KAFKA-14359: -- Assignee: Justine Olshan > Idempotent Producer continues to retry on OutOfOrderSequen

Re: [PR] KAFKA-15586: Clean shutdown detection - server side [kafka]

2024-03-18 Thread via GitHub
CalvinConfluent commented on PR #14706: URL: https://github.com/apache/kafka/pull/14706#issuecomment-2005099068 The test failures are irrelevant. -- 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

Re: [PR] KAFKA-16383: use public constructor [kafka]

2024-03-18 Thread via GitHub
chia7712 commented on code in PR #15556: URL: https://github.com/apache/kafka/pull/15556#discussion_r1529285548 ## connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectorValidationIntegrationTest.java: ## @@ -460,7 +460,7 @@ public static abstract class Abst

Re: [PR] KAFKA-16352: Txn may get get stuck in PrepareCommit or PrepareAbort s… [kafka]

2024-03-18 Thread via GitHub
jolshan commented on PR #15524: URL: https://github.com/apache/kafka/pull/15524#issuecomment-2004976461 I ran "until failure" on trunk and it took about 30 runs to fail. -- 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-16294: Add group protocol migration enabling config [kafka]

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

Re: [PR] KAFKA-16352: Txn may get get stuck in PrepareCommit or PrepareAbort s… [kafka]

2024-03-18 Thread via GitHub
artemlivshits commented on PR #15524: URL: https://github.com/apache/kafka/pull/15524#issuecomment-2004953080 LogDirFailureTest passed locally as well (probably just a flake) and it doesn't seem to use any of the code that I touched, so unrelated. -- This is an automated message from the

Re: [PR] KAFKA-15756: [2/3] Migrate existing integration tests to run old protocol in new coordinator [kafka]

2024-03-18 Thread via GitHub
kirktrue commented on code in PR #14675: URL: https://github.com/apache/kafka/pull/14675#discussion_r1529229638 ## core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala: ## @@ -89,6 +96,8 @@ class ConsumerBounceTest extends AbstractConsumerTest with Logging {

[jira] [Commented] (KAFKA-16217) Transactional producer stuck in IllegalStateException during close

2024-03-18 Thread Kirk True (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16217?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17828089#comment-17828089 ] Kirk True commented on KAFKA-16217: --- [~calvinliu]—I reassigned this to you as you've a

[jira] [Assigned] (KAFKA-16217) Transactional producer stuck in IllegalStateException during close

2024-03-18 Thread Kirk True (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16217?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Kirk True reassigned KAFKA-16217: - Assignee: Calvin Liu (was: Kirk True) > Transactional producer stuck in IllegalStateException

Re: [PR] KAFKA-16217: Stop the abort transaction try loop when closing producers [kafka]

2024-03-18 Thread via GitHub
kirktrue commented on PR #15541: URL: https://github.com/apache/kafka/pull/15541#issuecomment-2004890051 @CalvinConfluent—thanks for the PR! This PR doesn't have any unit tests to verify the new behavior. Would it be possible to migrate the test case from your _other PR_ (#15336) to _

Re: [PR] KAFKA-16374; High watermark updates should have a higher priority [kafka]

2024-03-18 Thread via GitHub
jeffkbkim commented on code in PR #15534: URL: https://github.com/apache/kafka/pull/15534#discussion_r1529191724 ## group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/CoordinatorRuntime.java: ## @@ -1191,30 +1212,37 @@ public void onHighWatermarkUpdated(

[PR] KAFKA-15517: Improve MirrorMaker logging in case of authorization errors [kafka]

2024-03-18 Thread via GitHub
wernerdv opened a new pull request, #15558: URL: https://github.com/apache/kafka/pull/15558 Log the list of topics for which an authorization error has been received when try to describe configs, along with the cluster alias. ### Committer Checklist (excluded from commit message) -

Re: [PR] MINOR; Log reason for deleting a kraft snapshot [kafka]

2024-03-18 Thread via GitHub
jsancio merged PR #15478: URL: https://github.com/apache/kafka/pull/15478 -- 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-16345: Optionally URL-encode clientID and clientSecret in authorization header [kafka]

2024-03-18 Thread via GitHub
kirktrue commented on code in PR #15475: URL: https://github.com/apache/kafka/pull/15475#discussion_r1529074359 ## clients/src/main/java/org/apache/kafka/common/config/SaslConfigs.java: ## @@ -192,6 +192,12 @@ public class SaslConfigs { + " be inspected for the stan

Re: [PR] KAFKA-15950: Serialize broker heartbeat requests [kafka]

2024-03-18 Thread via GitHub
junrao commented on code in PR #14903: URL: https://github.com/apache/kafka/pull/14903#discussion_r1526788841 ## core/src/main/scala/kafka/server/BrokerLifecycleManager.scala: ## @@ -551,9 +580,11 @@ class BrokerLifecycleManager( } private def scheduleNextCommunication(i

[jira] [Commented] (KAFKA-15282) Implement client support for KIP-848 client-side assignors

2024-03-18 Thread Kirk True (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15282?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17828075#comment-17828075 ] Kirk True commented on KAFKA-15282: --- [~zxcoccer]—you're certainly welcome to look at t

Re: [PR] KAFKA-14133: Move stateDirectory mock in TaskManagerTest to Mockito [kafka]

2024-03-18 Thread via GitHub
clolov commented on code in PR #15254: URL: https://github.com/apache/kafka/pull/15254#discussion_r1529018575 ## streams/src/test/java/org/apache/kafka/streams/processor/internals/TaskManagerTest.java: ## @@ -190,7 +181,7 @@ public class TaskManagerTest { @org.mockito.Moc

Re: [PR] KAFKA-16352: Txn may get get stuck in PrepareCommit or PrepareAbort s… [kafka]

2024-03-18 Thread via GitHub
jolshan commented on PR #15524: URL: https://github.com/apache/kafka/pull/15524#issuecomment-2004586672 @artemlivshits it may be worth checking if the test is failing on trunk as well. If so we can renew the JIRA to fix it 😅 -- This is an automated message from the Apache Git Service. To

Re: [PR] KAFKA-16369: Broker may not shut down when SocketServer fails to bind as Address already in use [kafka]

2024-03-18 Thread via GitHub
edoardocomar commented on PR #15530: URL: https://github.com/apache/kafka/pull/15530#issuecomment-2004581473 fix cherry picked to 3.6 and 3.7 branches -- 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

[jira] [Commented] (KAFKA-16369) Broker may not shut down when SocketServer fails to bind as Address already in use

2024-03-18 Thread Edoardo Comar (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16369?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17828067#comment-17828067 ] Edoardo Comar commented on KAFKA-16369: --- fix cherry picked to 3.6 and 3.7 > Broke

Re: [PR] KAFKA-16352: Txn may get get stuck in PrepareCommit or PrepareAbort s… [kafka]

2024-03-18 Thread via GitHub
jolshan commented on PR #15524: URL: https://github.com/apache/kafka/pull/15524#issuecomment-2004580064 > LogDirFailureTest > testIOExceptionDuringLogRoll(String) has been failing for a while -- there are some issues I tried to tackle with respect to it, but we ran into some issues. See ht

Re: [PR] KAFKA-16313: Offline group protocol migration (reopened) [kafka]

2024-03-18 Thread via GitHub
dongnuo123 commented on code in PR #15546: URL: https://github.com/apache/kafka/pull/15546#discussion_r1529010875 ## group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java: ## @@ -9298,6 +9298,120 @@ public void testOnConsumerGroupState

Re: [PR] KAFKA-16314: Introducing the AbortableTransactionException [kafka]

2024-03-18 Thread via GitHub
jolshan commented on code in PR #15486: URL: https://github.com/apache/kafka/pull/15486#discussion_r1529005304 ## core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala: ## @@ -3040,6 +3048,7 @@ class ReplicaManagerTest { transactionalId = transactionalId,

Re: [PR] KAFKA-16314: Introducing the AbortableTransactionException [kafka]

2024-03-18 Thread via GitHub
jolshan commented on code in PR #15486: URL: https://github.com/apache/kafka/pull/15486#discussion_r1528996611 ## clients/src/test/java/org/apache/kafka/clients/producer/internals/TransactionManagerTest.java: ## @@ -3523,6 +3524,128 @@ public void testForegroundInvalidStateTran

Re: [PR] KAFKA-16314: Introducing the AbortableTransactionException [kafka]

2024-03-18 Thread via GitHub
jolshan commented on code in PR #15486: URL: https://github.com/apache/kafka/pull/15486#discussion_r1528996611 ## clients/src/test/java/org/apache/kafka/clients/producer/internals/TransactionManagerTest.java: ## @@ -3523,6 +3524,128 @@ public void testForegroundInvalidStateTran

Re: [PR] KAFKA-16314: Introducing the AbortableTransactionException [kafka]

2024-03-18 Thread via GitHub
jolshan commented on PR #15486: URL: https://github.com/apache/kafka/pull/15486#issuecomment-2004546341 Hey @sjhajharia thanks for the updates. I think `ApiVersionErrorMapper` is also a bit tricky since the addPartitions change will not be related to errors at all. That's why I was thinking

Re: [PR] KAFKA-7663: Reprocessing on user added global stores restore [kafka]

2024-03-18 Thread via GitHub
wcarlson5 commented on PR #15414: URL: https://github.com/apache/kafka/pull/15414#issuecomment-2004539886 @mjsax I added some testing -- 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

Re: [PR] KAFKA-14585: Refactoring for moving the storage tool [kafka]

2024-03-18 Thread via GitHub
fvaleri commented on code in PR #15273: URL: https://github.com/apache/kafka/pull/15273#discussion_r1471706520 ## raft/src/main/java/org/apache/kafka/raft/RaftConfig.java: ## @@ -206,6 +245,27 @@ private static Integer parseVoterId(String idString) { } } +pri

[jira] [Updated] (KAFKA-16297) Race condition while promoting future replica can lead to partition unavailability.

2024-03-18 Thread Igor Soarez (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16297?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Igor Soarez updated KAFKA-16297: Description: KIP-858 proposed that when a directory failure occurs after changing the assignment

[jira] [Assigned] (KAFKA-16363) Storage crashes if dir is unavailable

2024-03-18 Thread Igor Soarez (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16363?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Igor Soarez reassigned KAFKA-16363: --- Assignee: (was: Igor Soarez) > Storage crashes if dir is unavailable >

Re: [PR] KAFKA-14752: Kafka examples improvements - processor changes [kafka]

2024-03-18 Thread via GitHub
fvaleri commented on code in PR #13516: URL: https://github.com/apache/kafka/pull/13516#discussion_r1528843407 ## examples/src/main/java/kafka/examples/ExactlyOnceMessageProcessor.java: ## @@ -81,111 +88,115 @@ public ExactlyOnceMessageProcessor(final String inputTopic,

Re: [PR] KAFKA-7663: Reprocessing on user added global stores restore [kafka]

2024-03-18 Thread via GitHub
wcarlson5 commented on code in PR #15414: URL: https://github.com/apache/kafka/pull/15414#discussion_r1528822628 ## streams/src/test/java/org/apache/kafka/streams/integration/GlobalStateReprocessTest.java: ## @@ -0,0 +1,205 @@ +/* + * Licensed to the Apache Software Foundation (

Re: [PR] KAFKA-7663: Reprocessing on user added global stores restore [kafka]

2024-03-18 Thread via GitHub
wcarlson5 commented on code in PR #15414: URL: https://github.com/apache/kafka/pull/15414#discussion_r1528810589 ## streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateManagerImpl.java: ## @@ -236,6 +252,91 @@ private List topicPartitionsForStore(final

Re: [PR] KAFKA-7663: Reprocessing on user added global stores restore [kafka]

2024-03-18 Thread via GitHub
wcarlson5 commented on code in PR #15414: URL: https://github.com/apache/kafka/pull/15414#discussion_r1528807717 ## streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordDeserializer.java: ## @@ -68,40 +68,49 @@ ConsumerRecord deserialize(final ProcessorConte

Re: [PR] KAFKA-16367; Full ConsumerGroupHeartbeat response must be sent when full request is received [kafka]

2024-03-18 Thread via GitHub
lianetm commented on code in PR #15533: URL: https://github.com/apache/kafka/pull/15533#discussion_r1528798734 ## group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java: ## @@ -1650,6 +1650,102 @@ public void testShouldThrowFencedInstan

Re: [PR] KAFKA-16367; Full ConsumerGroupHeartbeat response must be sent when full request is received [kafka]

2024-03-18 Thread via GitHub
lianetm commented on code in PR #15533: URL: https://github.com/apache/kafka/pull/15533#discussion_r1528793006 ## group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java: ## @@ -1190,10 +1190,11 @@ private CoordinatorResult consumerGr

Re: [PR] KAFKA-14752: Kafka examples improvements - processor changes [kafka]

2024-03-18 Thread via GitHub
gaoran10 commented on code in PR #13516: URL: https://github.com/apache/kafka/pull/13516#discussion_r1528650747 ## examples/src/main/java/kafka/examples/ExactlyOnceMessageProcessor.java: ## @@ -81,111 +88,115 @@ public ExactlyOnceMessageProcessor(final String inputTopic,

[jira] [Updated] (KAFKA-16375) Fix logic for discarding reconciliation if member rejoined

2024-03-18 Thread Lianet Magrans (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16375?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Lianet Magrans updated KAFKA-16375: --- Labels: client-transitions-issues kip-848-client-support (was: kip-848-client-support) > F

Re: [PR] KAFKA-14752: Kafka examples improvements - processor changes [kafka]

2024-03-18 Thread via GitHub
gaoran10 commented on code in PR #13516: URL: https://github.com/apache/kafka/pull/13516#discussion_r1528650747 ## examples/src/main/java/kafka/examples/ExactlyOnceMessageProcessor.java: ## @@ -81,111 +88,115 @@ public ExactlyOnceMessageProcessor(final String inputTopic,

Re: [PR] KAFKA-14752: Kafka examples improvements - processor changes [kafka]

2024-03-18 Thread via GitHub
gaoran10 commented on code in PR #13516: URL: https://github.com/apache/kafka/pull/13516#discussion_r1528650747 ## examples/src/main/java/kafka/examples/ExactlyOnceMessageProcessor.java: ## @@ -81,111 +88,115 @@ public ExactlyOnceMessageProcessor(final String inputTopic,

[jira] [Assigned] (KAFKA-15517) Improve MirrorMaker logging in case of authorization errors

2024-03-18 Thread Dmitry Werner (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15517?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Dmitry Werner reassigned KAFKA-15517: - Assignee: Dmitry Werner > Improve MirrorMaker logging in case of authorization errors >

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

2024-03-18 Thread Lianet Magrans (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15551?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17827990#comment-17827990 ] Lianet Magrans commented on KAFKA-15551: Also, given how tight the deadline is t

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

2024-03-18 Thread Lianet Magrans (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15551?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17827989#comment-17827989 ] Lianet Magrans commented on KAFKA-15551: Hey [~zxcoccer], thanks for jumping in!

Re: [PR] KAFKA-14133: Move stateDirectory mock in TaskManagerTest to Mockito [kafka]

2024-03-18 Thread via GitHub
cadonna commented on code in PR #15254: URL: https://github.com/apache/kafka/pull/15254#discussion_r1528601954 ## streams/src/test/java/org/apache/kafka/streams/processor/internals/TaskManagerTest.java: ## @@ -190,7 +181,7 @@ public class TaskManagerTest { @org.mockito.Mo

[jira] [Commented] (KAFKA-16369) Broker may not shut down when SocketServer fails to bind as Address already in use

2024-03-18 Thread Edoardo Comar (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16369?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17827976#comment-17827976 ] Edoardo Comar commented on KAFKA-16369: --- fix merged in trunk thanks [~showuon] >

[jira] [Updated] (KAFKA-16369) Broker may not shut down when SocketServer fails to bind as Address already in use

2024-03-18 Thread Edoardo Comar (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16369?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Edoardo Comar updated KAFKA-16369: -- Fix Version/s: 3.8.0 > Broker may not shut down when SocketServer fails to bind as Address alr

[jira] [Updated] (KAFKA-16369) Broker may not shut down when SocketServer fails to bind as Address already in use

2024-03-18 Thread Edoardo Comar (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16369?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Edoardo Comar updated KAFKA-16369: -- Fix Version/s: 3.6.2 3.7.1 > Broker may not shut down when SocketServer fai

[PR] use public constructor [kafka]

2024-03-18 Thread via GitHub
johnnychhsu opened a new pull request, #15556: URL: https://github.com/apache/kafka/pull/15556 ## Context This test failed in several PR, and from one of the failed build, there was an error log ``` [2024-03-12 13:58:12,744] ERROR Failed to discover HeaderConverter in classpath: Un

Re: [PR] KAFKA-14133: Move stateDirectory mock in TaskManagerTest to Mockito [kafka]

2024-03-18 Thread via GitHub
ijuma commented on code in PR #15254: URL: https://github.com/apache/kafka/pull/15254#discussion_r1528552125 ## streams/src/test/java/org/apache/kafka/streams/processor/internals/TaskManagerTest.java: ## @@ -190,7 +181,7 @@ public class TaskManagerTest { @org.mockito.Mock

[jira] [Created] (KAFKA-16383) fix flaky test IdentityReplicationIntegrationTest.testReplicateFromLatest()

2024-03-18 Thread Johnny Hsu (Jira)
Johnny Hsu created KAFKA-16383: -- Summary: fix flaky test IdentityReplicationIntegrationTest.testReplicateFromLatest() Key: KAFKA-16383 URL: https://issues.apache.org/jira/browse/KAFKA-16383 Project: Kafk

Re: [PR] [WIP] Splitting consumer tests [kafka]

2024-03-18 Thread via GitHub
lianetm closed pull request #15535: [WIP] Splitting consumer tests URL: https://github.com/apache/kafka/pull/15535 -- 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 unsubscrib

[jira] [Commented] (KAFKA-15951) MissingSourceTopicException should include topic names

2024-03-18 Thread sanghyeok An (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15951?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17827968#comment-17827968 ] sanghyeok An commented on KAFKA-15951: -- Hi, [~mjsax] ! May i take this issue and w

[PR] KIP-780: Support fine-grained compression options [kafka]

2024-03-18 Thread via GitHub
KevinZTW opened a new pull request, #1: URL: https://github.com/apache/kafka/pull/1 KIP: https://cwiki.apache.org/confluence/display/KAFKA/KIP-780%3A+Support+fine-grained+compression+options#KIP780:Supportfinegrainedcompressionoptions-Producer *More detailed description of you

Re: [PR] KAFKA-16318 : add javafoc for kafka metric [kafka]

2024-03-18 Thread via GitHub
johnnychhsu commented on code in PR #15483: URL: https://github.com/apache/kafka/pull/15483#discussion_r1528503006 ## clients/src/main/java/org/apache/kafka/common/metrics/KafkaMetric.java: ## @@ -78,6 +111,10 @@ public Measurable measurable() { } } +/** +

Re: [PR] KAFKA-16341: fix the LogValidator for non-compressed type [kafka]

2024-03-18 Thread via GitHub
johnnychhsu commented on code in PR #15476: URL: https://github.com/apache/kafka/pull/15476#discussion_r1528497039 ## core/src/test/scala/integration/kafka/admin/ListOffsetsIntegrationTest.scala: ## @@ -79,9 +79,34 @@ class ListOffsetsIntegrationTest extends KafkaServerTestHarn

Re: [PR] KAFKA-15062: Adding ppc64le build stage [kafka]

2024-03-18 Thread via GitHub
Vaibhav-Nazare commented on PR #13817: URL: https://github.com/apache/kafka/pull/13817#issuecomment-2003833677 Hi @mimaison additional jenkinsfile for ppc64le has been added , also as per my understanding some changes would be required at https://ci-builds.apache.org/job/Kafka/job/kafka/ to

Re: [PR] KAFKA-16341: fix the LogValidator for non-compressed type [kafka]

2024-03-18 Thread via GitHub
johnnychhsu commented on code in PR #15476: URL: https://github.com/apache/kafka/pull/15476#discussion_r1528477660 ## core/src/test/scala/integration/kafka/admin/ListOffsetsIntegrationTest.scala: ## @@ -79,9 +79,34 @@ class ListOffsetsIntegrationTest extends KafkaServerTestHarn

[jira] [Commented] (KAFKA-15538) Client support for java regex based subscription

2024-03-18 Thread Phuc Hong Tran (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15538?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17827950#comment-17827950 ] Phuc Hong Tran commented on KAFKA-15538: [~lianetm] [~kirktrue], I have a questi

Re: [PR] KAFKA-14133: Move stateDirectory mock in TaskManagerTest to Mockito [kafka]

2024-03-18 Thread via GitHub
clolov commented on code in PR #15254: URL: https://github.com/apache/kafka/pull/15254#discussion_r1528403432 ## streams/src/test/java/org/apache/kafka/streams/processor/internals/TaskManagerTest.java: ## @@ -785,7 +776,6 @@ public void shouldNotReturnStateUpdaterTasksInOwnedTas

Re: [PR] KAFKA-14133: Move stateDirectory mock in TaskManagerTest to Mockito [kafka]

2024-03-18 Thread via GitHub
clolov commented on PR #15254: URL: https://github.com/apache/kafka/pull/15254#issuecomment-2003726516 Heya @cadonna! This should be rebased, tests ought to be passing and all comments are addressed 😊. Let me know if there is something else you would suggest improving! -- This is an auto

Re: [PR] MINOR: Expose earliest local timestamp via the GetOffsetShell [kafka]

2024-03-18 Thread via GitHub
clolov commented on PR #14788: URL: https://github.com/apache/kafka/pull/14788#issuecomment-2003676807 Closing this in favour of https://issues.apache.org/jira/browse/KAFKA-15857 -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub

Re: [PR] MINOR: Expose earliest local timestamp via the GetOffsetShell [kafka]

2024-03-18 Thread via GitHub
clolov closed pull request #14788: MINOR: Expose earliest local timestamp via the GetOffsetShell URL: https://github.com/apache/kafka/pull/14788 -- 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 s

[jira] [Commented] (KAFKA-16322) Fix CVE-2023-50572 by updating jline from 3.22.0 to 3.25.1

2024-03-18 Thread Chia-Ping Tsai (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16322?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17827933#comment-17827933 ] Chia-Ping Tsai commented on KAFKA-16322: [~omkreddy] thanks for backporting this

Re: [PR] MINOR: Revert to Gradle 8.5 [kafka]

2024-03-18 Thread via GitHub
jlprat commented on PR #15553: URL: https://github.com/apache/kafka/pull/15553#issuecomment-2003653879 I managed to reproduce the issue with Gradle 8.6 once, but now that I'm trying to reproduce it again, it seems that incremental compilation works as expected (I tried it on `trunk` and mod

Re: [PR] MINOR: Update dependencies [kafka]

2024-03-18 Thread via GitHub
jlprat commented on code in PR #15404: URL: https://github.com/apache/kafka/pull/15404#discussion_r1528355399 ## gradle/dependencies.gradle: ## @@ -100,10 +100,10 @@ versions += [ commonsCli: "1.4", commonsValidator: "1.7", dropwizardMetrics: "4.1.12.1", - gradle: "8.5

[PR] MINOR: Update upgrade docs to refer 3.6.2 version [kafka]

2024-03-18 Thread via GitHub
omkreddy opened a new pull request, #15554: URL: https://github.com/apache/kafka/pull/15554 (no comment) -- 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-m

Re: [PR] KAFKA-16312, KAFKA-16185: Local epochs in reconciliation [kafka]

2024-03-18 Thread via GitHub
lucasbru merged PR #15511: URL: https://github.com/apache/kafka/pull/15511 -- 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-15417 Sloppy bug: moved the outerJounBreak-flags out of the loop [kafka]

2024-03-18 Thread via GitHub
VictorvandenHoven commented on code in PR #15510: URL: https://github.com/apache/kafka/pull/15510#discussion_r1527973416 ## streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoin.java: ## @@ -223,57 +224,51 @@ private void emitNonJoinedOuterRecords(

Re: [PR] KAFKA-15417 Sloppy bug: moved the outerJounBreak-flags out of the loop [kafka]

2024-03-18 Thread via GitHub
VictorvandenHoven commented on code in PR #15510: URL: https://github.com/apache/kafka/pull/15510#discussion_r1527968309 ## streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamOuterJoinTest.java: ## @@ -727,7 +801,7 @@ public void testWindowing() {

Re: [PR] KAFKA-15417 Sloppy bug: moved the outerJounBreak-flags out of the loop [kafka]

2024-03-18 Thread via GitHub
VictorvandenHoven commented on code in PR #15510: URL: https://github.com/apache/kafka/pull/15510#discussion_r1528271251 ## streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamOuterJoinTest.java: ## @@ -511,14 +511,88 @@ public void testGracePeriod() {

Re: [PR] KAFKA-15417 Sloppy bug: moved the outerJounBreak-flags out of the loop [kafka]

2024-03-18 Thread via GitHub
VictorvandenHoven commented on code in PR #15510: URL: https://github.com/apache/kafka/pull/15510#discussion_r1528270350 ## streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamOuterJoinTest.java: ## @@ -511,14 +511,88 @@ public void testGracePeriod() {

[jira] [Assigned] (KAFKA-16272) Update connect_distributed_test.py to support KIP-848’s group protocol config

2024-03-18 Thread Sagar Rao (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16272?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Sagar Rao reassigned KAFKA-16272: - Assignee: Sagar Rao > Update connect_distributed_test.py to support KIP-848’s group protocol co

Re: [PR] KAFKA-16369: Broker may not shut down when SocketServer fails to bind as Address already in use [kafka]

2024-03-18 Thread via GitHub
edoardocomar commented on PR #15530: URL: https://github.com/apache/kafka/pull/15530#issuecomment-2003461536 thanks @showuon ! -- 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-16369: Broker may not shut down when SocketServer fails to bind as Address already in use [kafka]

2024-03-18 Thread via GitHub
edoardocomar merged PR #15530: URL: https://github.com/apache/kafka/pull/15530 -- 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.ap

Re: [PR] KAFKA-16369: Broker may not shut down when SocketServer fails to bind as Address already in use [kafka]

2024-03-18 Thread via GitHub
edoardocomar commented on code in PR #15530: URL: https://github.com/apache/kafka/pull/15530#discussion_r1528223058 ## core/src/test/scala/unit/kafka/server/KafkaServerTest.scala: ## @@ -42,6 +43,24 @@ class KafkaServerTest extends QuorumTestHarness { TestUtils.shutdownServ

Re: [PR] KAFKA-16341: fix the LogValidator for non-compressed type [kafka]

2024-03-18 Thread via GitHub
showuon commented on code in PR #15476: URL: https://github.com/apache/kafka/pull/15476#discussion_r1528195441 ## core/src/test/scala/integration/kafka/admin/ListOffsetsIntegrationTest.scala: ## @@ -79,9 +79,34 @@ class ListOffsetsIntegrationTest extends KafkaServerTestHarness

Re: [PR] KAFKA-16073: [Tiered] Update localLogStartOffset before deleting segments in memory t… [kafka]

2024-03-18 Thread via GitHub
omkreddy commented on PR #15141: URL: https://github.com/apache/kafka/pull/15141#issuecomment-2003349984 If we want this to be 3.6.2 release, we need to merge the PR in next couple of days. -- This is an automated message from the Apache Git Service. To respond to the message, please log

[jira] [Updated] (KAFKA-16322) Fix CVE-2023-50572 by updating jline from 3.22.0 to 3.25.1

2024-03-18 Thread Manikumar (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16322?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar updated KAFKA-16322: -- Fix Version/s: 3.7.1 > Fix CVE-2023-50572 by updating jline from 3.22.0 to 3.25.1 > --

[jira] [Updated] (KAFKA-16322) Fix CVE-2023-50572 by updating jline from 3.22.0 to 3.25.1

2024-03-18 Thread Manikumar (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16322?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar updated KAFKA-16322: -- Fix Version/s: 3.6.2 > Fix CVE-2023-50572 by updating jline from 3.22.0 to 3.25.1 > --

[jira] [Updated] (KAFKA-16210) Upgrade jose4j to 0.9.4

2024-03-18 Thread Manikumar (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16210?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar updated KAFKA-16210: -- Fix Version/s: 3.6.2 > Upgrade jose4j to 0.9.4 > --- > > Key: KAFK

[jira] [Created] (KAFKA-16382) Kafka Streams drop NULL values after reset

2024-03-18 Thread Stanislav Spiridonov (Jira)
Stanislav Spiridonov created KAFKA-16382: Summary: Kafka Streams drop NULL values after reset Key: KAFKA-16382 URL: https://issues.apache.org/jira/browse/KAFKA-16382 Project: Kafka Is

Re: [PR] MINOR: Revert to Gradle 8.5 [kafka]

2024-03-18 Thread via GitHub
jlprat commented on PR #15553: URL: https://github.com/apache/kafka/pull/15553#issuecomment-2003177508 I accidentally committed the `core/data` files from some failed test run. @dajac feel free to review now. Thanks! -- This is an automated message from the Apache Git Service. To respon

Re: [PR] MINOR: Revert to Gradle 8.5 [kafka]

2024-03-18 Thread via GitHub
jlprat commented on PR #15553: URL: https://github.com/apache/kafka/pull/15553#issuecomment-2003170494 Oops, you are right @dajac I don't know what happened. I'll check -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use t

  1   2   >