Re: [PR] KAFKA-16209 : fetchSnapshot might return null if topic is created before v2.8 [kafka]

2024-03-05 Thread via GitHub
showuon commented on PR #15444: URL: https://github.com/apache/kafka/pull/15444#issuecomment-1980240072 Ah, you're right! @iit2009060 , I missed that! Will you open another PR to fix it? -- This is an automated message from the Apache Git Service. To respond to the message, please log on

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

2024-03-05 Thread via GitHub
Phuc-Hong-Tran commented on code in PR #15188: URL: https://github.com/apache/kafka/pull/15188#discussion_r1513877668 ## clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java: ## @@ -84,6 +85,9 @@ private enum SubscriptionType { /* the

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

2024-03-05 Thread via GitHub
Phuc-Hong-Tran commented on code in PR #15188: URL: https://github.com/apache/kafka/pull/15188#discussion_r1513877668 ## clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java: ## @@ -84,6 +85,9 @@ private enum SubscriptionType { /* the

Re: [PR] KAFKA-16152: Fix PlaintextConsumerTest.testStaticConsumerDetectsNewPartitionCreatedAfterRestart [kafka]

2024-03-05 Thread via GitHub
dajac commented on code in PR #15419: URL: https://github.com/apache/kafka/pull/15419#discussion_r1513841644 ## clients/src/main/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManager.java: ## @@ -546,8 +546,9 @@ public ConsumerGroupHeartbeatRequestData

Re: [PR] KAFKA-16209 : fetchSnapshot might return null if topic is created before v2.8 [kafka]

2024-03-05 Thread via GitHub
iit2009060 commented on PR #15444: URL: https://github.com/apache/kafka/pull/15444#issuecomment-1980040479 @showuon @chiacyu This has not fix the overall problem. It just moves the NullPointerException in the RemoteLogManager instead of ProducerStateManager.

Re: [PR] [WIP]KAFKA-15444: Native docker image [kafka]

2024-03-05 Thread via GitHub
github-actions[bot] commented on PR #14556: URL: https://github.com/apache/kafka/pull/14556#issuecomment-1980016226 This PR is being marked as stale since it has not had any activity in 90 days. If you would like to keep this PR alive, please ask a committer for review. If the PR has

Re: [PR] KAFKA-16345: optionally urlencode clientId and clientSecret in authorization header [kafka]

2024-03-05 Thread via GitHub
bachmanity1 commented on PR #15475: URL: https://github.com/apache/kafka/pull/15475#issuecomment-1980014397 Hi @kirktrue, thanks for the review! I've created a new KIP here

[PR] MINOR: Upgrade zookeeper 3.8.3 -> 3.8.4 [kafka]

2024-03-05 Thread via GitHub
KevinZTW opened a new pull request, #15480: URL: https://github.com/apache/kafka/pull/15480 upgrade Zookeeper from to 3.8.3 -> 3.8.4 ### Committer Checklist (excluded from commit message) - [ ] Verify design and implementation - [ ] Verify test coverage and CI build status -

[jira] [Commented] (KAFKA-16346) Fix flay MetricsTest.testMetrics

2024-03-05 Thread Chia-Ping Tsai (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16346?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17823831#comment-17823831 ] Chia-Ping Tsai commented on KAFKA-16346: The count is increased even though the value is zero,

[jira] [Assigned] (KAFKA-16346) Fix flay MetricsTest.testMetrics

2024-03-05 Thread PoAn Yang (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16346?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] PoAn Yang reassigned KAFKA-16346: - Assignee: PoAn Yang > Fix flay MetricsTest.testMetrics > > >

[jira] [Created] (KAFKA-16346) Fix flay MetricsTest.testMetrics

2024-03-05 Thread Chia-Ping Tsai (Jira)
Chia-Ping Tsai created KAFKA-16346: -- Summary: Fix flay MetricsTest.testMetrics Key: KAFKA-16346 URL: https://issues.apache.org/jira/browse/KAFKA-16346 Project: Kafka Issue Type: Bug

Re: [PR] KAFKA-16252: Fix the documentation and adjust the format [kafka]

2024-03-05 Thread via GitHub
chia7712 commented on code in PR #15473: URL: https://github.com/apache/kafka/pull/15473#discussion_r1513702636 ## connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ConnectMetrics.java: ## @@ -431,6 +432,7 @@ public interface LiteralSupplier { * @param args

Re: [PR] KAFKA-16252: Fix the documentation and adjust the format [kafka]

2024-03-05 Thread via GitHub
KevinZTW commented on code in PR #15473: URL: https://github.com/apache/kafka/pull/15473#discussion_r1513701081 ## connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ConnectMetrics.java: ## @@ -431,6 +432,7 @@ public interface LiteralSupplier { * @param args

Re: [PR] KAFKA-16345: optionally urlencode clientId and clientSecret in authorization header [kafka]

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

Re: [PR] KAFKA-16123: KStreamKStreamJoinProcessor does not drop late records. [kafka]

2024-03-05 Thread via GitHub
mjsax commented on PR #15189: URL: https://github.com/apache/kafka/pull/15189#issuecomment-1979900538 @florin-akermann -- I finally merged https://github.com/apache/kafka/pull/14426 -- can you rebase this PR and fixup tests so we can move forward with this PR? -- This is an automated

[jira] [Resolved] (KAFKA-15417) JoinWindow does not seem to work properly with a KStream - KStream - LeftJoin()

2024-03-05 Thread Matthias J. Sax (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15417?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax resolved KAFKA-15417. - Fix Version/s: 3.8.0 Resolution: Fixed > JoinWindow does not seem to work

Re: [PR] KAFKA-15417 flip joinSpuriousLookBackTimeMs and emit non-joined items [kafka]

2024-03-05 Thread via GitHub
mjsax commented on PR #14426: URL: https://github.com/apache/kafka/pull/14426#issuecomment-1979898854 Thanks for the fix! Merged to `trunk`. Really appreciate that you did push this through. Was more complicated than expected and took way to long to get finished. -- This is an

Re: [PR] KAFKA-15417 flip joinSpuriousLookBackTimeMs and emit non-joined items [kafka]

2024-03-05 Thread via GitHub
mjsax merged PR #14426: URL: https://github.com/apache/kafka/pull/14426 -- 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-16209 : fetchSnapshot might return null if topic is created before v2.8 [kafka]

2024-03-05 Thread via GitHub
showuon commented on PR #15444: URL: https://github.com/apache/kafka/pull/15444#issuecomment-1979894426 Thanks for the fix! -- 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] [Resolved] (KAFKA-16209) fetchSnapshot might return null if topic is created before v2.8

2024-03-05 Thread Luke Chen (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16209?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Luke Chen resolved KAFKA-16209. --- Fix Version/s: 3.8.0 3.7.1 Resolution: Fixed > fetchSnapshot might return

Re: [PR] KAFKA-16209 : fetchSnapshot might return null if topic is created before v2.8 [kafka]

2024-03-05 Thread via GitHub
showuon merged PR #15444: URL: https://github.com/apache/kafka/pull/15444 -- 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] [Commented] (KAFKA-16344) Internal topic mm2-offset-syncsinternal created with single partition is putting more load on the broker

2024-03-05 Thread Greg Harris (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16344?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17823817#comment-17823817 ] Greg Harris commented on KAFKA-16344: - Hi [~janardhanag], thanks for the ticket. At the current

Re: [PR] KAFKA-16342: fix getOffsetByMaxTimestamp for compressed records [kafka]

2024-03-05 Thread via GitHub
showuon commented on code in PR #15474: URL: https://github.com/apache/kafka/pull/15474#discussion_r1513659746 ## tools/src/test/java/org/apache/kafka/tools/GetOffsetShellTest.java: ## @@ -52,20 +55,30 @@ public class GetOffsetShellTest { private final int topicCount = 4;

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

2024-03-05 Thread via GitHub
CalvinConfluent commented on code in PR #15470: URL: https://github.com/apache/kafka/pull/15470#discussion_r1513643851 ## clients/src/main/java/org/apache/kafka/clients/admin/DescribeTopicsOptions.java: ## @@ -47,8 +49,32 @@ public DescribeTopicsOptions

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

2024-03-05 Thread via GitHub
CalvinConfluent commented on code in PR #15470: URL: https://github.com/apache/kafka/pull/15470#discussion_r1513646133 ## clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java: ## @@ -2276,6 +2415,11 @@ private Node leader(PartitionInfo partitionInfo) {

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

2024-03-05 Thread via GitHub
CalvinConfluent commented on code in PR #15470: URL: https://github.com/apache/kafka/pull/15470#discussion_r1513643851 ## clients/src/main/java/org/apache/kafka/clients/admin/DescribeTopicsOptions.java: ## @@ -47,8 +49,32 @@ public DescribeTopicsOptions

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

2024-03-05 Thread via GitHub
CalvinConfluent commented on code in PR #15470: URL: https://github.com/apache/kafka/pull/15470#discussion_r1513641333 ## tools/src/main/java/org/apache/kafka/tools/TopicCommand.java: ## @@ -799,6 +823,11 @@ public TopicCommandOptions(String[] args) { "if set

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

2024-03-05 Thread via GitHub
CalvinConfluent commented on code in PR #15470: URL: https://github.com/apache/kafka/pull/15470#discussion_r1513641009 ## clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java: ## @@ -1399,6 +1404,184 @@ public void testInvalidTopicNames() throws

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

2024-03-05 Thread via GitHub
CalvinConfluent commented on code in PR #15470: URL: https://github.com/apache/kafka/pull/15470#discussion_r1513640718 ## clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java: ## @@ -2190,6 +2201,117 @@ void handleFailure(Throwable throwable) {

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

2024-03-05 Thread via GitHub
CalvinConfluent commented on code in PR #15470: URL: https://github.com/apache/kafka/pull/15470#discussion_r1513640577 ## clients/src/main/java/org/apache/kafka/clients/admin/DescribeTopicsOptions.java: ## @@ -47,8 +49,32 @@ public DescribeTopicsOptions

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

2024-03-05 Thread via GitHub
CalvinConfluent commented on code in PR #15470: URL: https://github.com/apache/kafka/pull/15470#discussion_r1513640434 ## clients/src/main/java/org/apache/kafka/clients/admin/DescribeTopicsOptions.java: ## @@ -47,8 +49,32 @@ public DescribeTopicsOptions

Re: [PR] MINOR: Remove unused controlPlaneRequestProcessor in BrokerServer. [kafka]

2024-03-05 Thread via GitHub
appchemist commented on PR #15245: URL: https://github.com/apache/kafka/pull/15245#issuecomment-1979765706 @chia7712 sorry, I checked it late. I run the failed tests on my local too ```./gradlew cleanTest connect:mirror:test --tests MirrorConnectorsIntegrationExactlyOnceTest

Re: [PR] MINOR: Add read/write all operation [kafka]

2024-03-05 Thread via GitHub
chia7712 commented on code in PR #15462: URL: https://github.com/apache/kafka/pull/15462#discussion_r1513587296 ## group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorService.java: ## @@ -498,29 +497,17 @@ public CompletableFuture listGroups(

Re: [PR] KAFKA-16319: Divide DeleteTopics requests by leader node [kafka]

2024-03-05 Thread via GitHub
kirktrue commented on code in PR #15479: URL: https://github.com/apache/kafka/pull/15479#discussion_r1513571996 ## clients/src/main/java/org/apache/kafka/clients/admin/internals/DeleteRecordsHandler.java: ## @@ -79,15 +79,15 @@ public static SimpleAdminApiFuture newFuture(

[jira] [Updated] (KAFKA-15402) Performance regression on close consumer after upgrading to 3.5.0

2024-03-05 Thread Kirk True (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15402?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Kirk True updated KAFKA-15402: -- Labels: consumer-threading-refactor (was: ) > Performance regression on close consumer after

[jira] [Updated] (KAFKA-15402) Performance regression on close consumer after upgrading to 3.5.0

2024-03-05 Thread Kirk True (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15402?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Kirk True updated KAFKA-15402: -- Component/s: clients > Performance regression on close consumer after upgrading to 3.5.0 >

Re: [PR] KAFKA-16100: Add timeout to all the CompletableApplicationEvents [kafka]

2024-03-05 Thread via GitHub
kirktrue commented on code in PR #15455: URL: https://github.com/apache/kafka/pull/15455#discussion_r1513515432 ## clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitEvent.java: ## @@ -29,10 +30,19 @@ public abstract class CommitEvent extends

Re: [PR] KAFKA-14589 [3/4] Tests of ConsoleGroupCommand rewritten in java [kafka]

2024-03-05 Thread via GitHub
chia7712 commented on PR #15365: URL: https://github.com/apache/kafka/pull/15365#issuecomment-1979651000 @nizhikov I feel this PR is ready, and so please check (or list) the failed tests. If they are unconnected to this PR, I will merge it. -- This is an automated message from the Apache

Re: [PR] MINOR: Remove unused controlPlaneRequestProcessor in BrokerServer. [kafka]

2024-03-05 Thread via GitHub
chia7712 merged PR #15245: URL: https://github.com/apache/kafka/pull/15245 -- 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] MINOR: Remove unused controlPlaneRequestProcessor in BrokerServer. [kafka]

2024-03-05 Thread via GitHub
chia7712 commented on PR #15245: URL: https://github.com/apache/kafka/pull/15245#issuecomment-1979648508 run the failed tests on my local: ```sh ./gradlew cleanTest core:test --tests FetchRequestTestDowngrade --tests ProduceRequestTest --tests DynamicBrokerReconfigurationTest

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

2024-03-05 Thread via GitHub
kirktrue commented on code in PR #15470: URL: https://github.com/apache/kafka/pull/15470#discussion_r1513390603 ## clients/src/main/java/org/apache/kafka/clients/admin/DescribeTopicsOptions.java: ## @@ -47,8 +49,32 @@ public DescribeTopicsOptions

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

2024-03-05 Thread via GitHub
cmccabe merged PR #15435: URL: https://github.com/apache/kafka/pull/15435 -- 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] [Commented] (KAFKA-16319) Wrong broker destinations for DeleteRecords requests when more than one topic is involved and the topics/partitions are led by different brokers

2024-03-05 Thread AlexeyASF (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16319?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17823757#comment-17823757 ] AlexeyASF commented on KAFKA-16319: --- Great news, thank you very much for quick reaction! (y) > Wrong

[jira] [Commented] (KAFKA-16319) Wrong broker destinations for DeleteRecords requests when more than one topic is involved and the topics/partitions are led by different brokers

2024-03-05 Thread Andrew Schofield (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16319?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17823755#comment-17823755 ] Andrew Schofield commented on KAFKA-16319: -- No worries. My initial assessment was incorrect.

[PR] KAFKA-16319: Divide DeleteTopics requests by leader node [kafka]

2024-03-05 Thread via GitHub
AndrewJSchofield opened a new pull request, #15479: URL: https://github.com/apache/kafka/pull/15479 PR https://github.com/apache/kafka/pull/13760 introduced a problem with KafkaAdmin.deleteRecords. If the request acted on a set of topic-partitions which spanned multiple leader brokers, the

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

2024-03-05 Thread via GitHub
jsancio opened a new pull request, #15478: URL: https://github.com/apache/kafka/pull/15478 There are three reasons why KRaft would delete a snapshot. One, it is older than the retention time. Two, the total number of bytes between the log and the snapshot excess the configuration. Three,

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

2024-03-05 Thread via GitHub
dajac commented on code in PR #15470: URL: https://github.com/apache/kafka/pull/15470#discussion_r1513322953 ## clients/src/main/java/org/apache/kafka/clients/admin/DescribeTopicsOptions.java: ## @@ -47,8 +49,32 @@ public DescribeTopicsOptions

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

2024-03-05 Thread via GitHub
dajac commented on code in PR #15470: URL: https://github.com/apache/kafka/pull/15470#discussion_r1513322953 ## clients/src/main/java/org/apache/kafka/clients/admin/DescribeTopicsOptions.java: ## @@ -47,8 +49,32 @@ public DescribeTopicsOptions

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

2024-03-05 Thread via GitHub
AndrewJSchofield commented on code in PR #15470: URL: https://github.com/apache/kafka/pull/15470#discussion_r1513310569 ## clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java: ## @@ -2190,6 +2201,117 @@ void handleFailure(Throwable throwable) {

Re: [PR] KAFKA-14133: Move consumer mock in TaskManagerTest to Mockito - part 2 [kafka]

2024-03-05 Thread via GitHub
clolov commented on PR #15261: URL: https://github.com/apache/kafka/pull/15261#issuecomment-1979391796 Thanks for the review @cadonna! I will provide an updated version tomorrow morning! -- This is an automated message from the Apache Git Service. To respond to the message, please log on

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

2024-03-05 Thread via GitHub
cmccabe commented on code in PR #15435: URL: https://github.com/apache/kafka/pull/15435#discussion_r1513283747 ## core/src/test/scala/integration/kafka/server/KRaftClusterTest.scala: ## @@ -792,6 +792,43 @@ class KRaftClusterTest { } } + /** + * Test that setting

Re: [PR] KAFKA-15401 | Segment with corrupted index should not be tiered [kafka]

2024-03-05 Thread via GitHub
divijvaidya commented on code in PR #15472: URL: https://github.com/apache/kafka/pull/15472#discussion_r1513268220 ## core/src/main/java/kafka/log/remote/RemoteLogManager.java: ## @@ -707,6 +708,8 @@ public void copyLogSegmentsToRemote(UnifiedLog log) throws

[PR] [No Review] KAFKA-14563 part 1 [kafka]

2024-03-05 Thread via GitHub
CalvinConfluent opened a new pull request, #15477: URL: https://github.com/apache/kafka/pull/15477 Draft. -- 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,

[jira] [Commented] (KAFKA-16319) Wrong broker destinations for DeleteRecords requests when more than one topic is involved and the topics/partitions are led by different brokers

2024-03-05 Thread AlexeyASF (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16319?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17823723#comment-17823723 ] AlexeyASF commented on KAFKA-16319: --- ??How do you get it to do that? Do you have a small test program

[jira] [Commented] (KAFKA-16319) Wrong broker destinations for DeleteRecords requests when more than one topic is involved and the topics/partitions are led by different brokers

2024-03-05 Thread Andrew Schofield (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16319?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17823706#comment-17823706 ] Andrew Schofield commented on KAFKA-16319: -- I have reproduced it. Certainly fails like this on

Re: [PR] KAFKA-16322 : upgrade jline [kafka]

2024-03-05 Thread via GitHub
johnnychhsu commented on PR #15464: URL: https://github.com/apache/kafka/pull/15464#issuecomment-1979168636 the Jenkins pipeline (jdk8, scala 2.12) failed due to `Unable to connect to the child process`. I tried the same command in locall, run `./gradlew -PscalaVersion=2.12 clean check -x

[jira] [Commented] (KAFKA-14048) The Next Generation of the Consumer Rebalance Protocol

2024-03-05 Thread Aratz (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-14048?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17823698#comment-17823698 ] Aratz commented on KAFKA-14048: --- Where can I find the timeline for this work? Is there any? Thank you.

Re: [PR] MINOR: Add read/write all operation [kafka]

2024-03-05 Thread via GitHub
dajac commented on PR #15462: URL: https://github.com/apache/kafka/pull/15462#issuecomment-1979123455 @chia7712 Would you be interested in reviewing this one? -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL

[jira] [Commented] (KAFKA-16099) Handle timeouts for AsyncKafkaConsumer.commitSync

2024-03-05 Thread Aratz (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16099?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17823686#comment-17823686 ] Aratz commented on KAFKA-16099: --- Where can we find the corresponding PR? > Handle timeouts for

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

2024-03-05 Thread PoAn Yang (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16222?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17823675#comment-17823675 ] PoAn Yang commented on KAFKA-16222: --- I can reproduce the error. I will look into code tomorrow.

Re: [PR] KAFKA-16342: fix getOffsetByMaxTimestamp for compressed records [kafka]

2024-03-05 Thread via GitHub
ijuma commented on code in PR #15474: URL: https://github.com/apache/kafka/pull/15474#discussion_r1513013595 ## tools/src/test/java/org/apache/kafka/tools/GetOffsetShellTest.java: ## @@ -52,20 +55,30 @@ public class GetOffsetShellTest { private final int topicCount = 4;

Re: [PR] KAFKA-16293: Test log directory failure in Kraft [kafka]

2024-03-05 Thread via GitHub
pprovenzano commented on code in PR #15409: URL: https://github.com/apache/kafka/pull/15409#discussion_r1513010665 ## tests/kafkatest/tests/core/log_dir_failure_test.py: ## @@ -84,20 +84,25 @@ def __init__(self, test_context): self.num_consumers = 1 def

Re: [PR] KAFKA-15417 flip joinSpuriousLookBackTimeMs and emit non-joined items [kafka]

2024-03-05 Thread via GitHub
VictorvandenHoven commented on code in PR #14426: URL: https://github.com/apache/kafka/pull/14426#discussion_r1512985080 ## streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamOuterJoinTest.java: ## @@ -884,11 +886,13 @@ public void

Re: [PR] KAFKA-15417 flip joinSpuriousLookBackTimeMs and emit non-joined items [kafka]

2024-03-05 Thread via GitHub
VictorvandenHoven commented on code in PR #14426: URL: https://github.com/apache/kafka/pull/14426#discussion_r1512983679 ## streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamOuterJoinTest.java: ## @@ -438,13 +438,13 @@ public void testOrdering() {

Re: [PR] KAFKA-15417 flip joinSpuriousLookBackTimeMs and emit non-joined items [kafka]

2024-03-05 Thread via GitHub
VictorvandenHoven commented on code in PR #14426: URL: https://github.com/apache/kafka/pull/14426#discussion_r1512982523 ## streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamLeftJoinTest.java: ## @@ -436,6 +436,239 @@ public void

Re: [PR] KAFKA-15417 flip joinSpuriousLookBackTimeMs and emit non-joined items [kafka]

2024-03-05 Thread via GitHub
VictorvandenHoven commented on code in PR #14426: URL: https://github.com/apache/kafka/pull/14426#discussion_r1512981646 ## streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamLeftJoinTest.java: ## @@ -436,6 +436,239 @@ public void

Re: [PR] KAFKA-15417 flip joinSpuriousLookBackTimeMs and emit non-joined items [kafka]

2024-03-05 Thread via GitHub
VictorvandenHoven commented on code in PR #14426: URL: https://github.com/apache/kafka/pull/14426#discussion_r1512980857 ## streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamLeftJoinTest.java: ## @@ -436,6 +436,239 @@ public void

Re: [PR] KAFKA-16252: Fix the documentation and adjust the format [kafka]

2024-03-05 Thread via GitHub
KevinZTW commented on code in PR #15473: URL: https://github.com/apache/kafka/pull/15473#discussion_r1512973940 ## connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ConnectMetrics.java: ## @@ -30,6 +30,7 @@ import org.apache.kafka.common.utils.AppInfoParser;

Re: [PR] MINOR: parameterize group-id in GroupMetadataManagerTestContext [kafka]

2024-03-05 Thread via GitHub
chia7712 merged PR #15467: URL: https://github.com/apache/kafka/pull/15467 -- 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] [Commented] (KAFKA-16319) Wrong broker destinations for DeleteRecords requests when more than one topic is involved and the topics/partitions are led by different brokers

2024-03-05 Thread Andrew Schofield (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16319?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17823653#comment-17823653 ] Andrew Schofield commented on KAFKA-16319: -- [~alexeyasf] How do you get it to do that? Do you

Re: [PR] MINOR: parameterize group-id in GroupMetadataManagerTestContext [kafka]

2024-03-05 Thread via GitHub
dongnuo123 commented on code in PR #15467: URL: https://github.com/apache/kafka/pull/15467#discussion_r1512942687 ## group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTestContext.java: ## @@ -901,7 +901,7 @@ public RebalanceResult

Re: [PR] KAFKA-16252: Fix the documentation and adjust the format [kafka]

2024-03-05 Thread via GitHub
chia7712 commented on code in PR #15473: URL: https://github.com/apache/kafka/pull/15473#discussion_r1512938465 ## connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ConnectMetrics.java: ## @@ -431,6 +432,7 @@ public interface LiteralSupplier { * @param args

[jira] [Commented] (KAFKA-14679) Add new __consumer_offsets records

2024-03-05 Thread Aratz (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-14679?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17823645#comment-17823645 ] Aratz commented on KAFKA-14679: --- Okay found it, I think it is this one:

[jira] [Commented] (KAFKA-14679) Add new __consumer_offsets records

2024-03-05 Thread Aratz (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-14679?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17823644#comment-17823644 ] Aratz commented on KAFKA-14679: --- Hello, Is there any Github PR related to this *resolved* work? > Add

Re: [PR] KAFKA-16252: Fix the documentation and adjust the format [kafka]

2024-03-05 Thread via GitHub
chia7712 commented on code in PR #15473: URL: https://github.com/apache/kafka/pull/15473#discussion_r1512899702 ## connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ConnectMetrics.java: ## @@ -30,6 +30,7 @@ import org.apache.kafka.common.utils.AppInfoParser;

Re: [PR] KAFKA-16252: Fix the documentation and adjust the format [kafka]

2024-03-05 Thread via GitHub
KevinZTW commented on code in PR #15473: URL: https://github.com/apache/kafka/pull/15473#discussion_r1512897988 ## connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ConnectMetrics.java: ## @@ -431,6 +432,7 @@ public interface LiteralSupplier { * @param args

Re: [PR] KAFKA-16252: Fix the documentation and adjust the format [kafka]

2024-03-05 Thread via GitHub
KevinZTW commented on PR #15473: URL: https://github.com/apache/kafka/pull/15473#issuecomment-1978869931 > Could you run the website with the change and paste the screenshot? Thanks. Sure! ## Previous Version

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

2024-03-05 Thread via GitHub
johnnychhsu commented on PR #15476: URL: https://github.com/apache/kafka/pull/15476#issuecomment-1978864287 thanks for the quick review @chia7712 , sure let me address that -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and

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

2024-03-05 Thread via GitHub
chia7712 commented on PR #15476: URL: https://github.com/apache/kafka/pull/15476#issuecomment-1978858961 The fix is perfect. Please rewrite the test according to #15474. Putting all test cases together is more readable. -- This is an automated message from the Apache Git Service. To

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

2024-03-05 Thread via GitHub
johnnychhsu opened a new pull request, #15476: URL: https://github.com/apache/kafka/pull/15476 ## Context Previously in the LogValidator, the `offsetOfMaxTimestamp` depends on two parameter check 1. timestampType 2. batch.toMagic If the `timestampType` is `LOG_APPEND_TIME`, and

Re: [PR] KAFKA-16342: fix getOffsetByMaxTimestamp for compressed records [kafka]

2024-03-05 Thread via GitHub
showuon commented on code in PR #15474: URL: https://github.com/apache/kafka/pull/15474#discussion_r1512790426 ## clients/src/main/java/org/apache/kafka/common/record/MemoryRecordsBuilder.java: ## @@ -263,13 +262,8 @@ public RecordsInfo info() { } else if (maxTimestamp

Re: [PR] KAFKA-16234: Log directory failure re-creates partitions in another logdir automatically [kafka]

2024-03-05 Thread via GitHub
soarez commented on code in PR #15335: URL: https://github.com/apache/kafka/pull/15335#discussion_r1512748010 ## core/src/main/scala/kafka/server/metadata/BrokerMetadataPublisher.scala: ## @@ -289,13 +289,10 @@ class BrokerMetadataPublisher( try { // Start log

Re: [PR] KAFKA-16252: Fix the documentation and adjust the format [kafka]

2024-03-05 Thread via GitHub
showuon commented on code in PR #15473: URL: https://github.com/apache/kafka/pull/15473#discussion_r1512747215 ## connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ConnectMetrics.java: ## @@ -431,6 +432,7 @@ public interface LiteralSupplier { * @param args

Re: [PR] KAFKA-16252: Fix the documentation and adjust the format [kafka]

2024-03-05 Thread via GitHub
chia7712 commented on code in PR #15473: URL: https://github.com/apache/kafka/pull/15473#discussion_r1512687959 ## connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ConnectMetrics.java: ## @@ -431,6 +432,7 @@ public interface LiteralSupplier { * @param args

Re: [PR] KAFKA-16252: Fix the documentation and adjust the format [kafka]

2024-03-05 Thread via GitHub
KevinZTW commented on code in PR #15473: URL: https://github.com/apache/kafka/pull/15473#discussion_r1512669605 ## connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ConnectMetrics.java: ## @@ -431,6 +432,7 @@ public interface LiteralSupplier { * @param args

Re: [PR] KAFKA-16252: Fix the documentation and adjust the format [kafka]

2024-03-05 Thread via GitHub
KevinZTW commented on code in PR #15473: URL: https://github.com/apache/kafka/pull/15473#discussion_r1512665815 ## connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ConnectMetrics.java: ## @@ -431,6 +432,7 @@ public interface LiteralSupplier { * @param args

Re: [PR] KAFKA-16345: optionally urlencode clientId and clientSecret in authorization header [kafka]

2024-03-05 Thread via GitHub
bachmanity1 commented on PR #15475: URL: https://github.com/apache/kafka/pull/15475#issuecomment-1978544182 @kirktrue @mimaison can you have a look, please? Thanks! -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the

[PR] KAFKA-16345: optionally urlencode clientId and clientSecret in authorization header [kafka]

2024-03-05 Thread via GitHub
bachmanity1 opened a new pull request, #15475: URL: https://github.com/apache/kafka/pull/15475 When a client communicates with OIDC provider to retrieve an access token RFC-6749 says that clientID and clientSecret must be urlencoded in the authorization header. (see

Re: [PR] KAFKA-16342: fix getOffsetByMaxTimestamp for compressed records [kafka]

2024-03-05 Thread via GitHub
chia7712 commented on code in PR #15474: URL: https://github.com/apache/kafka/pull/15474#discussion_r1512657775 ## clients/src/main/java/org/apache/kafka/common/record/MemoryRecordsBuilder.java: ## @@ -263,13 +262,8 @@ public RecordsInfo info() { } else if

[jira] [Created] (KAFKA-16345) Optionally allow urlencoding clientId and clientSecret in authorization header

2024-03-05 Thread Nelson B. (Jira)
Nelson B. created KAFKA-16345: - Summary: Optionally allow urlencoding clientId and clientSecret in authorization header Key: KAFKA-16345 URL: https://issues.apache.org/jira/browse/KAFKA-16345 Project:

[jira] [Assigned] (KAFKA-16345) Optionally allow urlencoding clientId and clientSecret in authorization header

2024-03-05 Thread Nelson B. (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16345?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Nelson B. reassigned KAFKA-16345: - Assignee: Nelson B. > Optionally allow urlencoding clientId and clientSecret in authorization

[jira] [Assigned] (KAFKA-16341) Fix un-compressed records

2024-03-05 Thread Johnny Hsu (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16341?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Johnny Hsu reassigned KAFKA-16341: -- Assignee: Johnny Hsu > Fix un-compressed records > - > >

Re: [PR] KAFKA-16252: Fix the documentation and adjust the format [kafka]

2024-03-05 Thread via GitHub
chia7712 commented on code in PR #15473: URL: https://github.com/apache/kafka/pull/15473#discussion_r1512617796 ## connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ConnectMetrics.java: ## @@ -431,6 +432,7 @@ public interface LiteralSupplier { * @param args

Re: [PR] KAFKA-14589 [3/4] Tests of ConsoleGroupCommand rewritten in java [kafka]

2024-03-05 Thread via GitHub
nizhikov commented on code in PR #15365: URL: https://github.com/apache/kafka/pull/15365#discussion_r1512610273 ## tools/src/test/java/org/apache/kafka/tools/consumer/group/SaslClientsWithInvalidCredentialsTest.java: ## @@ -0,0 +1,191 @@ +/* + * Licensed to the Apache Software

Re: [PR] KAFKA-14589 [3/4] Tests of ConsoleGroupCommand rewritten in java [kafka]

2024-03-05 Thread via GitHub
nizhikov commented on code in PR #15365: URL: https://github.com/apache/kafka/pull/15365#discussion_r1512609597 ## tools/src/test/java/org/apache/kafka/tools/consumer/group/SaslClientsWithInvalidCredentialsTest.java: ## @@ -0,0 +1,191 @@ +/* + * Licensed to the Apache Software

Re: [PR] KAFKA-16342: fix getOffsetByMaxTimestamp for compressed records [kafka]

2024-03-05 Thread via GitHub
showuon commented on code in PR #15474: URL: https://github.com/apache/kafka/pull/15474#discussion_r1512607158 ## tools/src/test/java/org/apache/kafka/tools/GetOffsetShellTest.java: ## @@ -333,7 +382,7 @@ private void assertExitCodeIsOne(String... args) { } private

Re: [PR] KAFKA-16342: fix getOffsetByMaxTimestamp for compressed records [kafka]

2024-03-05 Thread via GitHub
showuon commented on PR #15474: URL: https://github.com/apache/kafka/pull/15474#issuecomment-1978460395 @chia7712 @ijuma @hachikuji , please take a look. Thanks. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL

Re: [PR] KAFKA-16342: fix getOffsetByMaxTimestamp for compressed records [kafka]

2024-03-05 Thread via GitHub
showuon commented on code in PR #15474: URL: https://github.com/apache/kafka/pull/15474#discussion_r1512604852 ## storage/src/main/java/org/apache/kafka/storage/internals/log/LogValidator.java: ## @@ -379,8 +381,11 @@ public ValidationResult

Re: [PR] KAFKA-14589 [3/4] Tests of ConsoleGroupCommand rewritten in java [kafka]

2024-03-05 Thread via GitHub
chia7712 commented on code in PR #15365: URL: https://github.com/apache/kafka/pull/15365#discussion_r1512597183 ## tools/src/test/java/org/apache/kafka/tools/consumer/group/SaslClientsWithInvalidCredentialsTest.java: ## @@ -0,0 +1,191 @@ +/* + * Licensed to the Apache Software

[PR] KAFKA-16342: fix getOffsetByMaxTimestamp for compressed records [kafka]

2024-03-05 Thread via GitHub
showuon opened a new pull request, #15474: URL: https://github.com/apache/kafka/pull/15474 Fix `getOffsetByMaxTimestamp` for compressed records. This PR adds: 1. For inPlaceAssignment case, compute the correct offset for maxTimestamp when traversing the batch records, and set to

  1   2   >