[jira] [Updated] (KAFKA-16251) Fenced member should not send heartbeats while waiting for onPartitionsLost to complete

2024-02-20 Thread Kirk True (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16251?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Kirk True updated KAFKA-16251: -- Priority: Critical (was: Major) > Fenced member should not send heartbeats while waiting for

[jira] [Updated] (KAFKA-15561) Client support for new SubscriptionPattern based subscription

2024-02-20 Thread Kirk True (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15561?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Kirk True updated KAFKA-15561: -- Priority: Major (was: Critical) > Client support for new SubscriptionPattern based subscription >

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

2024-02-20 Thread Kirk True (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15538?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Kirk True updated KAFKA-15538: -- Priority: Blocker (was: Critical) > Client support for java regex based subscription >

Re: [PR] KAFKA-16286; Notify listener of latest leader and epoch [kafka]

2024-02-20 Thread via GitHub
jsancio commented on code in PR #15397: URL: https://github.com/apache/kafka/pull/15397#discussion_r1496389863 ## raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java: ## @@ -2958,6 +2958,59 @@ public void testHandleCommitCallbackFiresInCandidateState() throws

[PR] KAFKA-16286; Notify listener of latest leader and epoch [kafka]

2024-02-20 Thread via GitHub
jsancio opened a new pull request, #15397: URL: https://github.com/apache/kafka/pull/15397 KRaft was only notifying listeners of the latest leader and epoch when the replica transition to a new state. This can result in the listener never getting notified if the registration happened after

Re: [PR] MINOR: remove unnecessary logging [kafka]

2024-02-20 Thread via GitHub
mjsax commented on code in PR #15396: URL: https://github.com/apache/kafka/pull/15396#discussion_r1496379447 ## streams/src/main/java/org/apache/kafka/streams/state/internals/AbstractRocksDBSegmentedBytesStore.java: ## @@ -264,7 +264,6 @@ public void put(final Bytes key,

Re: [PR] MINOR: remove unnecessary logging [kafka]

2024-02-20 Thread via GitHub
kpatelatwork commented on code in PR #15396: URL: https://github.com/apache/kafka/pull/15396#discussion_r1496373291 ## streams/src/main/java/org/apache/kafka/streams/state/internals/AbstractRocksDBSegmentedBytesStore.java: ## @@ -264,7 +264,6 @@ public void put(final Bytes key,

Re: [PR] KAFKA-16265: KIP-994 (Part 1) Minor Enhancements to ListTransactionsRequest [kafka]

2024-02-20 Thread via GitHub
yyu1993 commented on code in PR #15384: URL: https://github.com/apache/kafka/pull/15384#discussion_r1496360067 ## clients/src/main/java/org/apache/kafka/clients/admin/ListTransactionsOptions.java: ## @@ -81,11 +87,16 @@ public Set filteredProducerIds() { return

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

2024-02-20 Thread via GitHub
CalvinConfluent commented on code in PR #15265: URL: https://github.com/apache/kafka/pull/15265#discussion_r1496355495 ## clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java: ## @@ -2129,63 +2167,183 @@ private Map> handleDescribeTopicsByNames(f

Re: [PR] KAFKA-16206 Fix unnecessary topic config deletion during ZK migration [kafka]

2024-02-20 Thread via GitHub
ahuang98 commented on PR #14206: URL: https://github.com/apache/kafka/pull/14206#issuecomment-1954880081 Thanks @mimaison, looks like this is because I split out the TopicsImageTest changes into a separate PR. The migration tests depend on the `DELTA1_RECORDS` defined there so I moved over

[PR] MINOR: remove unnecessary logging [kafka]

2024-02-20 Thread via GitHub
mjsax opened a new pull request, #15396: URL: https://github.com/apache/kafka/pull/15396 We already record dropping record via metrics and logging at WARN level is too noise. This PR removes the unnecessary logging. -- This is an automated message from the Apache Git Service. To

Re: [PR] KAFKA-16265: KIP-994 (Part 1) Minor Enhancements to ListTransactionsRequest [kafka]

2024-02-20 Thread via GitHub
yyu1993 commented on code in PR #15384: URL: https://github.com/apache/kafka/pull/15384#discussion_r1496339739 ## tools/src/test/java/org/apache/kafka/tools/TransactionsCommandTest.java: ## @@ -187,14 +187,25 @@ private void testDescribeProducers(

[jira] [Updated] (KAFKA-16286) KRaft doesn't always notify listener of latest leader

2024-02-20 Thread Jira
[ https://issues.apache.org/jira/browse/KAFKA-16286?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] José Armando García Sancio updated KAFKA-16286: --- Description: If a listener registers with RaftClient after the

[jira] [Created] (KAFKA-16286) KRaft doesn't always notify listener of latest leader

2024-02-20 Thread Jira
José Armando García Sancio created KAFKA-16286: -- Summary: KRaft doesn't always notify listener of latest leader Key: KAFKA-16286 URL: https://issues.apache.org/jira/browse/KAFKA-16286

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

2024-02-20 Thread via GitHub
artemlivshits commented on code in PR #15265: URL: https://github.com/apache/kafka/pull/15265#discussion_r1495047191 ## clients/src/main/java/org/apache/kafka/clients/admin/DescribeTopicsResult.java: ## @@ -36,28 +38,38 @@ public class DescribeTopicsResult { private final

[jira] [Assigned] (KAFKA-14747) FK join should record discarded subscription responses

2024-02-20 Thread Ayoub Omari (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-14747?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ayoub Omari reassigned KAFKA-14747: --- Assignee: Ayoub Omari (was: Koma Zhang) > FK join should record discarded subscription

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

2024-02-20 Thread via GitHub
AyoubOm opened a new pull request, #15395: URL: https://github.com/apache/kafka/pull/15395 *As described in KAFKA-14747, we are not recording discarded FK join responses in case of receiving a join response for an old record (whose hash value has changed in the left table). This PR adds

[jira] [Assigned] (KAFKA-16111) Implement tests for tricky rebalance callback scenarios

2024-02-20 Thread Kirk True (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16111?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Kirk True reassigned KAFKA-16111: - Assignee: Lucas Brutschy > Implement tests for tricky rebalance callback scenarios >

[jira] [Assigned] (KAFKA-16285) Make group metadata available when a new assignment is set in async Kafka consumer

2024-02-20 Thread Kirk True (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16285?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Kirk True reassigned KAFKA-16285: - Assignee: Bruno Cadonna > Make group metadata available when a new assignment is set in async

[jira] [Commented] (KAFKA-14747) FK join should record discarded subscription responses

2024-02-20 Thread Ayoub Omari (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-14747?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17818919#comment-17818919 ] Ayoub Omari commented on KAFKA-14747: - Picking this up since [~kma] didn't respond :) > FK join

Re: [PR] KAFKA-16033: Commit retry logic fixes [kafka]

2024-02-20 Thread via GitHub
lianetm commented on PR #15357: URL: https://github.com/apache/kafka/pull/15357#issuecomment-1954655740 Thanks for the comments @lucasbru , all addressed. -- 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-16194: Do not return records from poll if group metadata unknown [kafka]

2024-02-20 Thread via GitHub
cadonna commented on code in PR #15369: URL: https://github.com/apache/kafka/pull/15369#discussion_r1496180609 ## clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java: ## @@ -727,6 +732,17 @@ public ConsumerRecords poll(final Duration

Re: [PR] KAFKA-16033: Commit retry logic fixes [kafka]

2024-02-20 Thread via GitHub
lianetm commented on code in PR #15357: URL: https://github.com/apache/kafka/pull/15357#discussion_r1496179477 ## clients/src/main/java/org/apache/kafka/clients/consumer/internals/CommitRequestManager.java: ## @@ -1147,15 +1202,15 @@ List drain(final long currentTimeMs) {

Re: [PR] KAFKA-16194: Do not return records from poll if group metadata unknown [kafka]

2024-02-20 Thread via GitHub
cadonna commented on code in PR #15369: URL: https://github.com/apache/kafka/pull/15369#discussion_r1496167714 ## clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java: ## @@ -727,6 +732,17 @@ public ConsumerRecords poll(final Duration

Re: [PR] KAFKA-16194: Do not return records from poll if group metadata unknown [kafka]

2024-02-20 Thread via GitHub
cadonna commented on code in PR #15369: URL: https://github.com/apache/kafka/pull/15369#discussion_r1496159485 ## clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java: ## @@ -708,14 +709,18 @@ public ConsumerRecords poll(final Duration

Re: [PR] KAFKA-16194: Do not return records from poll if group metadata unknown [kafka]

2024-02-20 Thread via GitHub
cadonna commented on code in PR #15369: URL: https://github.com/apache/kafka/pull/15369#discussion_r1496159485 ## clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java: ## @@ -708,14 +709,18 @@ public ConsumerRecords poll(final Duration

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

2024-02-20 Thread Justine Olshan (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16282?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17818903#comment-17818903 ] Justine Olshan commented on KAFKA-16282: I am also happy to review the KIP and PRs :)  > Allow

Re: [PR] KAFKA-16265: KIP-994 (Part 1) Minor Enhancements to ListTransactionsRequest [kafka]

2024-02-20 Thread via GitHub
jolshan commented on code in PR #15384: URL: https://github.com/apache/kafka/pull/15384#discussion_r1496132259 ## core/src/main/scala/kafka/coordinator/transaction/TransactionCoordinator.scala: ## @@ -278,12 +278,13 @@ class TransactionCoordinator(txnConfig: TransactionConfig,

Re: [PR] KAFKA-16033: Commit retry logic fixes [kafka]

2024-02-20 Thread via GitHub
lianetm commented on code in PR #15357: URL: https://github.com/apache/kafka/pull/15357#discussion_r1496101641 ## clients/src/main/java/org/apache/kafka/clients/consumer/internals/CommitRequestManager.java: ## @@ -204,126 +205,315 @@ private static long findMinTime(final

Re: [PR] KAFKA-16033: Commit retry logic fixes [kafka]

2024-02-20 Thread via GitHub
lianetm commented on code in PR #15357: URL: https://github.com/apache/kafka/pull/15357#discussion_r1496091256 ## clients/src/main/java/org/apache/kafka/clients/consumer/internals/CommitRequestManager.java: ## @@ -719,26 +846,27 @@ NetworkClientDelegate.UnsentRequest

Re: [PR] KAFKA-16033: Commit retry logic fixes [kafka]

2024-02-20 Thread via GitHub
lianetm commented on code in PR #15357: URL: https://github.com/apache/kafka/pull/15357#discussion_r1496065426 ## clients/src/main/java/org/apache/kafka/clients/consumer/internals/CommitRequestManager.java: ## @@ -204,126 +205,315 @@ private static long findMinTime(final

Re: [PR] KAFKA-16033: Commit retry logic fixes [kafka]

2024-02-20 Thread via GitHub
lianetm commented on code in PR #15357: URL: https://github.com/apache/kafka/pull/15357#discussion_r1496060942 ## clients/src/main/java/org/apache/kafka/clients/consumer/internals/CommitRequestManager.java: ## @@ -204,126 +205,315 @@ private static long findMinTime(final

Re: [PR] KAFKA-16033: Commit retry logic fixes [kafka]

2024-02-20 Thread via GitHub
lianetm commented on code in PR #15357: URL: https://github.com/apache/kafka/pull/15357#discussion_r1496049319 ## clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java: ## @@ -149,8 +148,14 @@ private void process(final

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

2024-02-20 Thread Hector Geraldino (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16223?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17818856#comment-17818856 ] Hector Geraldino commented on KAFKA-16223: -- One thing that worked for me when migrating the

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

2024-02-20 Thread via GitHub
msn-tldr commented on PR #15385: URL: https://github.com/apache/kafka/pull/15385#issuecomment-1954448982 All the Jenkins test failures are in different tests, which are already known to have flaky history. -- This is an automated message from the Apache Git Service. To respond to the

Re: [PR] KAFKA-16194: Do not return records from poll if group metadata unknown [kafka]

2024-02-20 Thread via GitHub
cadonna commented on code in PR #15369: URL: https://github.com/apache/kafka/pull/15369#discussion_r1495859702 ## clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java: ## @@ -1467,6 +1483,7 @@ public void unsubscribe() { }

Re: [PR] KAFKA-16284: Fix performance regression in RocksDB [kafka]

2024-02-20 Thread via GitHub
lucasbru commented on PR #15393: URL: https://github.com/apache/kafka/pull/15393#issuecomment-1954244163 @cadonna could you take a look? -- 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] KAFKA-16194: Do not return records from poll if group metadata unknown [kafka]

2024-02-20 Thread via GitHub
lucasbru commented on code in PR #15369: URL: https://github.com/apache/kafka/pull/15369#discussion_r1495811911 ## clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java: ## @@ -98,6 +99,8 @@ import

[PR] KAFKA-16284: Fix performance regression in RocksDB [kafka]

2024-02-20 Thread via GitHub
lucasbru opened a new pull request, #15393: URL: https://github.com/apache/kafka/pull/15393 A performance regression introduced in commit 5bc3aa428067dff1f2b9075ff5d1351fb05d4b10 reduces the write performance in RocksDB by ~3x. The bug is that we fail to pass the `WriteOptions` that

[jira] [Assigned] (KAFKA-16284) Performance regression in RocksDB

2024-02-20 Thread Lucas Brutschy (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16284?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Lucas Brutschy reassigned KAFKA-16284: -- Assignee: Lucas Brutschy > Performance regression in RocksDB >

Re: [PR] KAFKA-12744: Breaking change dependency upgrade: "argparse4j" 0.7.0 -->> 0.9.0 [kafka]

2024-02-20 Thread via GitHub
dejan2609 commented on PR #10626: URL: https://github.com/apache/kafka/pull/10626#issuecomment-1954154725 Hi @mimaison ! I will return to this once Kafka drops support for Java 8. Reference JIRA ticket: **_KIP-750: Drop support for Java 8 in Kafka 4.0 (deprecate in 3.0)_**

[jira] [Updated] (KAFKA-16285) Make group metadata available when a new assignment is set in async Kafka consumer

2024-02-20 Thread Bruno Cadonna (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16285?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Bruno Cadonna updated KAFKA-16285: -- Description: Currently, the new async Kafka consumer sends an event from the background

[jira] [Created] (KAFKA-16285) Make group metadata available when a new assignment is set in async Kafka consumer

2024-02-20 Thread Bruno Cadonna (Jira)
Bruno Cadonna created KAFKA-16285: - Summary: Make group metadata available when a new assignment is set in async Kafka consumer Key: KAFKA-16285 URL: https://issues.apache.org/jira/browse/KAFKA-16285

[jira] [Commented] (KAFKA-16283) RoundRobinPartitioner will only send to half of the partitions in a topic

2024-02-20 Thread Luke Chen (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16283?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17818794#comment-17818794 ] Luke Chen commented on KAFKA-16283: --- [~alivshits], any thoughts on this? > RoundRobinPartitioner will

[jira] [Created] (KAFKA-16284) Performance regression in RocksDB

2024-02-20 Thread Lucas Brutschy (Jira)
Lucas Brutschy created KAFKA-16284: -- Summary: Performance regression in RocksDB Key: KAFKA-16284 URL: https://issues.apache.org/jira/browse/KAFKA-16284 Project: Kafka Issue Type: Task

Re: [PR] [WIP] KAFKA-13566: producer exponential backoff implementation for KIP-580 [kafka]

2024-02-20 Thread via GitHub
mimaison commented on PR #11646: URL: https://github.com/apache/kafka/pull/11646#issuecomment-1954090264 Done in https://github.com/apache/kafka/pull/14111, closing this PR -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and

Re: [PR] [WIP] KAFKA-13566: producer exponential backoff implementation for KIP-580 [kafka]

2024-02-20 Thread via GitHub
mimaison closed pull request #11646: [WIP] KAFKA-13566: producer exponential backoff implementation for KIP-580 URL: https://github.com/apache/kafka/pull/11646 -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL

Re: [PR] [WIP] KAFKA-13567: adminClient exponential backoff implementation for KIP-580 [kafka]

2024-02-20 Thread via GitHub
mimaison closed pull request #11647: [WIP] KAFKA-13567: adminClient exponential backoff implementation for KIP-580 URL: https://github.com/apache/kafka/pull/11647 -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL

Re: [PR] [WIP] KAFKA-13567: adminClient exponential backoff implementation for KIP-580 [kafka]

2024-02-20 Thread via GitHub
mimaison commented on PR #11647: URL: https://github.com/apache/kafka/pull/11647#issuecomment-1954090079 Done in https://github.com/apache/kafka/pull/14111, closing this PR -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and

Re: [PR] Add kafka topic sync offset lag metrics by JMX [kafka]

2024-02-20 Thread via GitHub
mimaison commented on PR #11387: URL: https://github.com/apache/kafka/pull/11387#issuecomment-1954086721 This is being done by KIP-971, so closing this PR. -- 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] Add kafka topic sync offset lag metrics by JMX [kafka]

2024-02-20 Thread via GitHub
mimaison closed pull request #11387: Add kafka topic sync offset lag metrics by JMX URL: https://github.com/apache/kafka/pull/11387 -- 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-15561: Client support for new SubscriptionPattern based subscription [kafka]

2024-02-20 Thread via GitHub
cadonna commented on code in PR #15188: URL: https://github.com/apache/kafka/pull/15188#discussion_r1495579840 ## clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java: ## @@ -1730,6 +1744,21 @@ private void subscribeInternal(Pattern pattern,

Re: [PR] KAFKA-12891: KIP-749 Add --files and --file-separator options to the ConsoleProducer (WIP) [kafka]

2024-02-20 Thread via GitHub
mimaison closed pull request #10889: KAFKA-12891: KIP-749 Add --files and --file-separator options to the ConsoleProducer (WIP) URL: https://github.com/apache/kafka/pull/10889 -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and

Re: [PR] KAFKA-12891: KIP-749 Add --files and --file-separator options to the ConsoleProducer (WIP) [kafka]

2024-02-20 Thread via GitHub
mimaison commented on PR #10889: URL: https://github.com/apache/kafka/pull/10889#issuecomment-1954080126 KIP-749 was never voted and is now abandoned so I'll close this PR. Feel free to restart the discussion on the KIP if it's still something you want to do. -- This is an automated

[jira] [Updated] (KAFKA-16283) RoundRobinPartitioner will only send to half of the partitions in a topic

2024-02-20 Thread Luke Chen (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16283?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Luke Chen updated KAFKA-16283: -- Description: When using `org.apache.kafka.clients.producer.RoundRobinPartitioner`, we expect data

Re: [PR] KAFKA-12744: Breaking change dependency upgrade: "argparse4j" 0.7.0 -->> 0.9.0 [kafka]

2024-02-20 Thread via GitHub
mimaison commented on PR #10626: URL: https://github.com/apache/kafka/pull/10626#issuecomment-1954070991 We can't upgrade to argparse 0.9.0 now as it requires Java 9. Closing this PR for now, we can revisit it once Kafka drops support for Java 8. -- This is an automated message from the

[jira] [Updated] (KAFKA-16283) RoundRobinPartitioner will only send to half of the partitions in a topic

2024-02-20 Thread Luke Chen (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16283?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Luke Chen updated KAFKA-16283: -- Description: When using `org.apache.kafka.clients.producer.RoundRobinPartitioner`, we expect data

Re: [PR] KAFKA-12744: Breaking change dependency upgrade: "argparse4j" 0.7.0 -->> 0.9.0 [kafka]

2024-02-20 Thread via GitHub
mimaison closed pull request #10626: KAFKA-12744: Breaking change dependency upgrade: "argparse4j" 0.7.0 -->> 0.9.0 URL: https://github.com/apache/kafka/pull/10626 -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL

[jira] [Created] (KAFKA-16283) RoundRobinPartitioner will only send to half of the partitions in a topic

2024-02-20 Thread Luke Chen (Jira)
Luke Chen created KAFKA-16283: - Summary: RoundRobinPartitioner will only send to half of the partitions in a topic Key: KAFKA-16283 URL: https://issues.apache.org/jira/browse/KAFKA-16283 Project: Kafka

[jira] [Updated] (KAFKA-16283) RoundRobinPartitioner will only send to half of the partitions in a topic

2024-02-20 Thread Luke Chen (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16283?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Luke Chen updated KAFKA-16283: -- Description: When using `org.apache.kafka.clients.producer.RoundRobinPartitioner`, we expect data

Re: [PR] [KAFKA-10718][Kafka Connect]add config settting, skip record when enc… [kafka]

2024-02-20 Thread via GitHub
mimaison commented on PR #9592: URL: https://github.com/apache/kafka/pull/9592#issuecomment-1954057264 This proposed a public API change so this requires a KIP: https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals Considering the age of this PR, I'll close it

Re: [PR] [KAFKA-10718][Kafka Connect]add config settting, skip record when enc… [kafka]

2024-02-20 Thread via GitHub
mimaison closed pull request #9592: [KAFKA-10718][Kafka Connect]add config settting, skip record when enc… URL: https://github.com/apache/kafka/pull/9592 -- 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

Re: [PR] KAFKA-10325: KIP-649 implementation [kafka]

2024-02-20 Thread via GitHub
mimaison commented on PR #9101: URL: https://github.com/apache/kafka/pull/9101#issuecomment-1954051810 KIP-649 was never voted and is now abandoned so I'll close this PR. Feel free to restart the discussion on the KIP if it's still something you want to do. -- This is an automated

Re: [PR] KAFKA-10325: KIP-649 implementation [kafka]

2024-02-20 Thread via GitHub
mimaison closed pull request #9101: KAFKA-10325: KIP-649 implementation URL: https://github.com/apache/kafka/pull/9101 -- 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

Re: [PR] KAFKA-10325: KIP-649 implementation [kafka]

2024-02-20 Thread via GitHub
mimaison commented on PR #9101: URL: https://github.com/apache/kafka/pull/9101#issuecomment-1954051581 KIP-649 was never voted and is now abandoned so I'll close this PR. Feel free to restart the discussion on the KIP if it's still something you want to do. -- This is an automated

Re: [PR] KAFKA-10369 [WIP] KIP-655 implementation [kafka]

2024-02-20 Thread via GitHub
mimaison commented on PR #9210: URL: https://github.com/apache/kafka/pull/9210#issuecomment-1954049633 KIP-655 was never voted and is now abandoned so I'll close this PR. Feel free to restart the discussion on the KIP if it's still something you want to do. -- This is an automated

Re: [PR] KAFKA-10369 [WIP] KIP-655 implementation [kafka]

2024-02-20 Thread via GitHub
mimaison closed pull request #9210: KAFKA-10369 [WIP] KIP-655 implementation URL: https://github.com/apache/kafka/pull/9210 -- 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

Re: [PR] KAFKA-10281: [WIP] Add log compression analysis tool KIP-640 [kafka]

2024-02-20 Thread via GitHub
mimaison commented on PR #9193: URL: https://github.com/apache/kafka/pull/9193#issuecomment-1954047088 Considering KIP-640 was never voted and is now abandoned, I'll close this PR. Feel free to restart the discussion on the KIP if it's still something you want to do. -- This is an

Re: [PR] KAFKA-10281: [WIP] Add log compression analysis tool KIP-640 [kafka]

2024-02-20 Thread via GitHub
mimaison closed pull request #9193: KAFKA-10281: [WIP] Add log compression analysis tool KIP-640 URL: https://github.com/apache/kafka/pull/9193 -- 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] KAFKA-10116: GraalVM native-image prototype [kafka]

2024-02-20 Thread via GitHub
mimaison commented on PR #8830: URL: https://github.com/apache/kafka/pull/8830#issuecomment-1954044169 This is now being done via KIP-974. Considering the age of this PR, I'll close it. -- This is an automated message from the Apache Git Service. To respond to the message, please log on

Re: [PR] KAFKA-10116: GraalVM native-image prototype [kafka]

2024-02-20 Thread via GitHub
mimaison closed pull request #8830: KAFKA-10116: GraalVM native-image prototype URL: https://github.com/apache/kafka/pull/8830 -- 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.

[jira] [Updated] (KAFKA-16281) Possible IllegalState with KIP-996

2024-02-20 Thread Jack Vanlightly (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16281?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jack Vanlightly updated KAFKA-16281: Description: I have a TLA+ model of KIP-996 (pre-vote) and I have identified an

Re: [PR] KAFKA-9800: [KIP-580] Client Exponential Backoff Implementation [kafka]

2024-02-20 Thread via GitHub
mimaison closed pull request #8846: KAFKA-9800: [KIP-580] Client Exponential Backoff Implementation URL: https://github.com/apache/kafka/pull/8846 -- 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-9800: [KIP-580] Client Exponential Backoff Implementation [kafka]

2024-02-20 Thread via GitHub
mimaison commented on PR #8846: URL: https://github.com/apache/kafka/pull/8846#issuecomment-1954041148 Done in https://github.com/apache/kafka/pull/14111, closing this PR -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and

Re: [PR] java 8 code level update [kafka]

2024-02-20 Thread via GitHub
mimaison closed pull request #6454: java 8 code level update URL: https://github.com/apache/kafka/pull/6454 -- 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,

Re: [PR] java 8 code level update [kafka]

2024-02-20 Thread via GitHub
mimaison commented on PR #6454: URL: https://github.com/apache/kafka/pull/6454#issuecomment-1954032896 It looks like most of these changes have been since the PR was originally opened. Considering its age I'm going to close it. -- This is an automated message from the Apache Git Service.

Re: [PR] KIP-228 Negative record timestamp support [kafka]

2024-02-20 Thread via GitHub
mimaison commented on PR #5072: URL: https://github.com/apache/kafka/pull/5072#issuecomment-1954026617 The KIP was never voted and is currently abandoned. So I'll close this PR for now, feel free to restart the discussion on the KIP if it's still something you want. -- This is an

Re: [PR] KIP-228 Negative record timestamp support [kafka]

2024-02-20 Thread via GitHub
mimaison closed pull request #5072: KIP-228 Negative record timestamp support URL: https://github.com/apache/kafka/pull/5072 -- 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.

[jira] [Updated] (KAFKA-16281) Possible IllegalState with KIP-996

2024-02-20 Thread Jack Vanlightly (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16281?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jack Vanlightly updated KAFKA-16281: Summary: Possible IllegalState with KIP-996 (was: Probable IllegalState possible with

Re: [PR] KAFKA-2111: Add help arguments and required fields [kafka]

2024-02-20 Thread via GitHub
mimaison commented on PR #3605: URL: https://github.com/apache/kafka/pull/3605#issuecomment-1954006952 This PR is very old and the tools are being rewritten in Java, so I'll close this PR. -- This is an automated message from the Apache Git Service. To respond to the message, please log

Re: [PR] KAFKA-2111: Add help arguments and required fields [kafka]

2024-02-20 Thread via GitHub
mimaison closed pull request #3605: KAFKA-2111: Add help arguments and required fields URL: https://github.com/apache/kafka/pull/3605 -- 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] Update KafkaLog4jAppender.java adding SaslMechanism as an option [kafka]

2024-02-20 Thread via GitHub
mimaison commented on PR #3476: URL: https://github.com/apache/kafka/pull/3476#issuecomment-1954004851 Considering the age of this PR and that KIP-719 to deprecate the log4j appender has been voted, I'll close this PR. -- This is an automated message from the Apache Git Service. To

Re: [PR] Update KafkaLog4jAppender.java adding SaslMechanism as an option [kafka]

2024-02-20 Thread via GitHub
mimaison closed pull request #3476: Update KafkaLog4jAppender.java adding SaslMechanism as an option URL: https://github.com/apache/kafka/pull/3476 -- 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-3663 : Implements KIP-59 for a kafka-brokers.sh command [kafka]

2024-02-20 Thread via GitHub
mimaison commented on PR #1539: URL: https://github.com/apache/kafka/pull/1539#issuecomment-1953997528 KIP-59 has never been voted and is abandoned so closing this PR. Feel free to restart the KIP discussion if it's still something you want to do. -- This is an automated message from the

Re: [PR] KAFKA-3663 : Implements KIP-59 for a kafka-brokers.sh command [kafka]

2024-02-20 Thread via GitHub
mimaison closed pull request #1539: KAFKA-3663 : Implements KIP-59 for a kafka-brokers.sh command URL: https://github.com/apache/kafka/pull/1539 -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the

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

2024-02-20 Thread Luke Chen (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16282?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Luke Chen updated KAFKA-16282: -- Description: Currently, when using `kafka-get-offsets.sh` to get the offset by time, we have these

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

2024-02-20 Thread Luke Chen (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16282?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17818759#comment-17818759 ] Luke Chen commented on KAFKA-16282: --- Great! Thanks [~ahmedsobeh]! Let me know if you need any help. >

[jira] [Commented] (KAFKA-15302) Stale value returned when using store.all() in punctuation function.

2024-02-20 Thread Ondrej Cervinka (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15302?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17818755#comment-17818755 ] Ondrej Cervinka commented on KAFKA-15302: - Hello, Deleting from a state store while iterating

Re: [PR] MINIOR: Remove accidentally logs [kafka]

2024-02-20 Thread via GitHub
mimaison merged PR #15371: URL: https://github.com/apache/kafka/pull/15371 -- 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-15561: Client support for new SubscriptionPattern based subscription [kafka]

2024-02-20 Thread via GitHub
Phuc-Hong-Tran commented on PR #15188: URL: https://github.com/apache/kafka/pull/15188#issuecomment-1953901380 @cadonna I'll see if there is a way to go around with not using the Google library to check regex validity (finger-crossed!) -- This is an automated message from the Apache Git

Re: [PR] KAFKA-15561: Client support for new SubscriptionPattern based subscription [kafka]

2024-02-20 Thread via GitHub
cadonna commented on PR #15188: URL: https://github.com/apache/kafka/pull/15188#issuecomment-1953893744 > > > I was wondering whether we should introduce a new error code to signal to the user that the regular expression is invalid. Otherwise, we would have to use the invalid request

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

2024-02-20 Thread Ahmed Sobeh (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16282?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17818729#comment-17818729 ] Ahmed Sobeh edited comment on KAFKA-16282 at 2/20/24 10:14 AM: --- Hi

Re: [PR] KAFKA-15561: Client support for new SubscriptionPattern based subscription [kafka]

2024-02-20 Thread via GitHub
Phuc-Hong-Tran commented on PR #15188: URL: https://github.com/apache/kafka/pull/15188#issuecomment-1953879987 > > I was wondering whether we should introduce a new error code to signal to the user that the regular expression is invalid. Otherwise, we would have to use the invalid request

Re: [PR] KAFKA-15561: Client support for new SubscriptionPattern based subscription [kafka]

2024-02-20 Thread via GitHub
cadonna commented on PR #15188: URL: https://github.com/apache/kafka/pull/15188#issuecomment-1953872828 > I was wondering whether we should introduce a new error code to signal to the user that the regular expression is invalid. Otherwise, we would have to use the invalid request exception

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

2024-02-20 Thread Ahmed Sobeh (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16282?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17818729#comment-17818729 ] Ahmed Sobeh commented on KAFKA-16282: - Hi [~showuon]! I will be picking this up. I will start on the

[jira] [Updated] (KAFKA-16243) Idle kafka-console-consumer with new consumer group protocol preemptively leaves group

2024-02-20 Thread Lucas Brutschy (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16243?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Lucas Brutschy updated KAFKA-16243: --- Fix Version/s: 3.8.0 > Idle kafka-console-consumer with new consumer group protocol

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

2024-02-20 Thread Ahmed Sobeh (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16282?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ahmed Sobeh reassigned KAFKA-16282: --- Assignee: Ahmed Sobeh > Allow to get last stable offset (LSO) in kafka-get-offsets.sh >

Re: [PR] KAFKA-16243: Make sure that we do not exceed max poll interval inside poll [kafka]

2024-02-20 Thread via GitHub
lucasbru merged PR #15372: URL: https://github.com/apache/kafka/pull/15372 -- 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] IGNORE: flaky testing [kafka]

2024-02-20 Thread via GitHub
lucasbru closed pull request #14780: IGNORE: flaky testing URL: https://github.com/apache/kafka/pull/14780 -- 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,

Re: [PR] KAFKA-15561: Client support for new SubscriptionPattern based subscription [kafka]

2024-02-20 Thread via GitHub
cadonna commented on PR #15188: URL: https://github.com/apache/kafka/pull/15188#issuecomment-1953815656 > > > @cadonna @lianetm, since we're supporting for both subscribe method using java.util.regex.Pattern and SubscriptionPattern, I think we should throw a illegal heartbeat exeption when

<    1   2   3   >