Re: [PR] KAFKA-15853: Move Sasl and SSL configs out of core [kafka]

2024-04-19 Thread via GitHub
omkreddy commented on code in PR #15656: URL: https://github.com/apache/kafka/pull/15656#discussion_r1573153281 ## server/src/main/java/org/apache/kafka/server/config/KafkaSecurityConfigs.java: ## @@ -0,0 +1,239 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under

Re: [PR] KAFKA-16211: Inconsistent config values in CreateTopicsResult and DescribeConfigsResult [kafka]

2024-04-19 Thread via GitHub
infantlikesprogramming commented on PR #15696: URL: https://github.com/apache/kafka/pull/15696#issuecomment-2067538939 @chia7712 Thanks for the reply. I have tried the following code and received the results. Each time I run the code, the `DescribeTopicsResult` gives a different

[jira] [Commented] (KAFKA-16588) broker shutdown hangs when `log.segment.delete.delay.ms` is zero

2024-04-19 Thread PoAn Yang (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16588?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17839170#comment-17839170 ] PoAn Yang commented on KAFKA-16588: --- Hi [~chia7712], I'm interested in this. May I assign to myself?

[jira] [Resolved] (KAFKA-15585) DescribeTopic API

2024-04-19 Thread Calvin Liu (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15585?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Calvin Liu resolved KAFKA-15585. Resolution: Fixed > DescribeTopic API > - > > Key: KAFKA-15585 >

Re: [PR] KAFKA-16568: JMH Benchmarks for Server Side Rebalances [kafka]

2024-04-19 Thread via GitHub
rreddy-22 commented on code in PR #15717: URL: https://github.com/apache/kafka/pull/15717#discussion_r1573016246 ## jmh-benchmarks/src/main/java/org/apache/kafka/jmh/assignor/ServerSideAssignorBenchmark.java: ## @@ -0,0 +1,252 @@ +/* + * Licensed to the Apache Software

Re: [PR] KAFKA-16568: JMH Benchmarks for Server Side Rebalances [kafka]

2024-04-19 Thread via GitHub
rreddy-22 commented on code in PR #15717: URL: https://github.com/apache/kafka/pull/15717#discussion_r1573016609 ## jmh-benchmarks/src/main/java/org/apache/kafka/jmh/assignor/ServerSideAssignorBenchmark.java: ## @@ -0,0 +1,252 @@ +/* + * Licensed to the Apache Software

Re: [PR] KAFKA-16568: JMH Benchmarks for Server Side Rebalances [kafka]

2024-04-19 Thread via GitHub
rreddy-22 commented on code in PR #15717: URL: https://github.com/apache/kafka/pull/15717#discussion_r1573016246 ## jmh-benchmarks/src/main/java/org/apache/kafka/jmh/assignor/ServerSideAssignorBenchmark.java: ## @@ -0,0 +1,252 @@ +/* + * Licensed to the Apache Software

Re: [PR] KAFKA-14226: feat(connect:transform): Introduce FieldPath abstraction [kafka]

2024-04-19 Thread via GitHub
jeqo commented on code in PR #15379: URL: https://github.com/apache/kafka/pull/15379#discussion_r1572995864 ## connect/transforms/src/main/java/org/apache/kafka/connect/transforms/field/MultiFieldPaths.java: ## @@ -0,0 +1,581 @@ +/* + * Licensed to the Apache Software

Re: [PR] MINOR: Fix io-[wait-]ratio metrics description [kafka]

2024-04-19 Thread via GitHub
emitskevich-blp commented on PR #15722: URL: https://github.com/apache/kafka/pull/15722#issuecomment-2067299812 > we should verify the non-deprecated metrics should have correct doc which is not marked as "deprecated". Also, that is what you try to fix, right? Correct, this is the

[jira] [Created] (KAFKA-16589) Consider removing `ClusterInstance#createAdminClient` since callers are not sure whether they need to call close

2024-04-19 Thread Chia-Ping Tsai (Jira)
Chia-Ping Tsai created KAFKA-16589: -- Summary: Consider removing `ClusterInstance#createAdminClient` since callers are not sure whether they need to call close Key: KAFKA-16589 URL:

Re: [PR] MINOR: example.com moved [kafka]

2024-04-19 Thread via GitHub
akatona84 commented on code in PR #15758: URL: https://github.com/apache/kafka/pull/15758#discussion_r1572917082 ## clients/src/test/java/org/apache/kafka/clients/ClientUtilsTest.java: ## @@ -61,7 +61,7 @@ public void testParseAndValidateAddressesWithReverseLookup() {

Re: [PR] KAFKA-16483: migrate DeleteOffsetsConsumerGroupCommandIntegrationTest to use ClusterTestExtensions [kafka]

2024-04-19 Thread via GitHub
chia7712 commented on code in PR #15679: URL: https://github.com/apache/kafka/pull/15679#discussion_r1572912365 ## tools/src/test/java/org/apache/kafka/tools/consumer/group/DeleteOffsetsConsumerGroupCommandIntegrationTest.java: ## @@ -16,7 +16,15 @@ */ package

[jira] [Commented] (KAFKA-16082) Broker recreates reassigned partition after logdir failure

2024-04-19 Thread Chia-Ping Tsai (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16082?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17839107#comment-17839107 ] Chia-Ping Tsai commented on KAFKA-16082: [~gnarula] Could you file PR for branch 3.7? > Broker

Re: [PR] KAFKA-16082: Avoid resuming future replica if current replica is in the same directory [kafka]

2024-04-19 Thread via GitHub
chia7712 merged PR #15136: URL: https://github.com/apache/kafka/pull/15136 -- 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-16082: Avoid resuming future replica if current replica is in the same directory [kafka]

2024-04-19 Thread via GitHub
chia7712 commented on PR #15136: URL: https://github.com/apache/kafka/pull/15136#issuecomment-2067228882 the failed `testParseAndValidateAddressesWithReverseLookup` is traced by #15758. will merge it -- This is an automated message from the Apache Git Service. To respond to the message,

Re: [PR] KAFKA-15853: Move KafkaConfig log properties and docs out of core [kafka]

2024-04-19 Thread via GitHub
chia7712 merged PR #15569: URL: https://github.com/apache/kafka/pull/15569 -- 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-16552: Create an internal config to control InitialTaskDelayMs in LogManager to speed up tests [kafka]

2024-04-19 Thread via GitHub
chia7712 commented on PR #15719: URL: https://github.com/apache/kafka/pull/15719#issuecomment-2067224991 @brandboat please fix the conflicts, 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 above to go

Re: [PR] KAFKA-15853: Move KafkaConfig log properties and docs out of core [kafka]

2024-04-19 Thread via GitHub
chia7712 commented on PR #15569: URL: https://github.com/apache/kafka/pull/15569#issuecomment-2067222576 The failed test `testParseAndValidateAddressesWithReverseLookup` will get fixed by #15758. I will merge this PR -- This is an automated message from the Apache Git Service. To respond

Re: [PR] MINOR: example.com moved [kafka]

2024-04-19 Thread via GitHub
chia7712 commented on code in PR #15758: URL: https://github.com/apache/kafka/pull/15758#discussion_r1572897153 ## clients/src/test/java/org/apache/kafka/clients/ClientUtilsTest.java: ## @@ -61,7 +61,7 @@ public void testParseAndValidateAddressesWithReverseLookup() {

Re: [PR] KAFKA-16539 Fix IncrementalAlterConfigs during ZK migration [kafka]

2024-04-19 Thread via GitHub
mumrah commented on PR #15744: URL: https://github.com/apache/kafka/pull/15744#issuecomment-2067204988 Updated to include a CheckOp on the `/controller` ZNode. We don't both using the controller epoch since it is not straightforward to consistently read the controller and controller epoch

Re: [PR] KAFKA-16507 Add raw record into RecordDeserialisationException [kafka]

2024-04-19 Thread via GitHub
AndrewJSchofield commented on code in PR #15691: URL: https://github.com/apache/kafka/pull/15691#discussion_r1572845716 ## clients/src/main/java/org/apache/kafka/clients/consumer/internals/CompletedFetch.java: ## @@ -311,25 +312,33 @@ ConsumerRecord parseRecord(Deserializers

[jira] [Commented] (KAFKA-16493) Avoid unneeded subscription regex check if metadata version unchanged

2024-04-19 Thread Phuc Hong Tran (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16493?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17839100#comment-17839100 ] Phuc Hong Tran commented on KAFKA-16493: [~lianetm] I’ll come back to this ticket this weekend

[jira] [Commented] (KAFKA-16493) Avoid unneeded subscription regex check if metadata version unchanged

2024-04-19 Thread Lianet Magrans (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16493?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17839096#comment-17839096 ] Lianet Magrans commented on KAFKA-16493: Hey [~phuctran], any progress on this one? Even though

[jira] [Updated] (KAFKA-16493) Avoid unneeded subscription regex check if metadata version unchanged

2024-04-19 Thread Lianet Magrans (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16493?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Lianet Magrans updated KAFKA-16493: --- Fix Version/s: 3.8.0 (was: 4.0.0) > Avoid unneeded subscription

Re: [PR] KAFKA-16539 Fix IncrementalAlterConfigs during ZK migration [kafka]

2024-04-19 Thread via GitHub
mumrah commented on code in PR #15744: URL: https://github.com/apache/kafka/pull/15744#discussion_r1572801229 ## core/src/main/scala/kafka/zk/KafkaZkClient.scala: ## @@ -467,13 +470,33 @@ class KafkaZkClient private[zk] (zooKeeperClient: ZooKeeperClient, isSecure: Boo *

[jira] [Resolved] (KAFKA-16486) Integrate metric measurability changes in metrics collector

2024-04-19 Thread Matthias J. Sax (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16486?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax resolved KAFKA-16486. - Fix Version/s: 3.8.0 Resolution: Done > Integrate metric measurability changes

Re: [PR] KAFKA-16486: Integrate KIP-1019 measurability changes (KIP-714) [kafka]

2024-04-19 Thread via GitHub
mjsax merged PR #15682: URL: https://github.com/apache/kafka/pull/15682 -- 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-16539 Fix IncrementalAlterConfigs during ZK migration [kafka]

2024-04-19 Thread via GitHub
mumrah commented on code in PR #15744: URL: https://github.com/apache/kafka/pull/15744#discussion_r1572780325 ## core/src/main/scala/kafka/zk/KafkaZkClient.scala: ## @@ -467,13 +470,33 @@ class KafkaZkClient private[zk] (zooKeeperClient: ZooKeeperClient, isSecure: Boo *

Re: [PR] KAFKA-16539 Fix IncrementalAlterConfigs during ZK migration [kafka]

2024-04-19 Thread via GitHub
mumrah commented on code in PR #15744: URL: https://github.com/apache/kafka/pull/15744#discussion_r1572777334 ## core/src/main/scala/kafka/zk/KafkaZkClient.scala: ## @@ -467,13 +470,33 @@ class KafkaZkClient private[zk] (zooKeeperClient: ZooKeeperClient, isSecure: Boo *

Re: [PR] KAFKA-16539 Fix IncrementalAlterConfigs during ZK migration [kafka]

2024-04-19 Thread via GitHub
mumrah commented on code in PR #15744: URL: https://github.com/apache/kafka/pull/15744#discussion_r1572776248 ## core/src/test/scala/integration/kafka/zk/ZkMigrationIntegrationTest.scala: ## @@ -950,16 +980,47 @@ class ZkMigrationIntegrationTest {

Re: [PR] KAFKA-16568: JMH Benchmarks for Server Side Rebalances [kafka]

2024-04-19 Thread via GitHub
rreddy-22 commented on code in PR #15717: URL: https://github.com/apache/kafka/pull/15717#discussion_r1572735673 ## jmh-benchmarks/src/main/java/org/apache/kafka/jmh/assignor/ServerSideAssignorBenchmark.java: ## @@ -0,0 +1,252 @@ +/* + * Licensed to the Apache Software

Re: [PR] KAFKA-16568: JMH Benchmarks for Server Side Rebalances [kafka]

2024-04-19 Thread via GitHub
rreddy-22 commented on code in PR #15717: URL: https://github.com/apache/kafka/pull/15717#discussion_r1572729047 ## jmh-benchmarks/src/main/java/org/apache/kafka/jmh/assignor/TargetAssignmentBuilderBenchmark.java: ## @@ -0,0 +1,202 @@ +/* + * Licensed to the Apache Software

Re: [PR] KAFKA-16568: JMH Benchmarks for Server Side Rebalances [kafka]

2024-04-19 Thread via GitHub
rreddy-22 commented on code in PR #15717: URL: https://github.com/apache/kafka/pull/15717#discussion_r1572726906 ## jmh-benchmarks/src/main/java/org/apache/kafka/jmh/assignor/ServerSideAssignorBenchmark.java: ## @@ -0,0 +1,266 @@ +/* + * Licensed to the Apache Software

Re: [PR] KAFKA-16539 Fix IncrementalAlterConfigs during ZK migration [kafka]

2024-04-19 Thread via GitHub
cmccabe commented on code in PR #15744: URL: https://github.com/apache/kafka/pull/15744#discussion_r1572722514 ## core/src/main/scala/kafka/zk/KafkaZkClient.scala: ## @@ -467,13 +470,33 @@ class KafkaZkClient private[zk] (zooKeeperClient: ZooKeeperClient, isSecure: Boo *

Re: [PR] KAFKA-16539 Fix IncrementalAlterConfigs during ZK migration [kafka]

2024-04-19 Thread via GitHub
cmccabe commented on code in PR #15744: URL: https://github.com/apache/kafka/pull/15744#discussion_r1572717868 ## core/src/main/scala/kafka/zk/KafkaZkClient.scala: ## @@ -467,13 +470,33 @@ class KafkaZkClient private[zk] (zooKeeperClient: ZooKeeperClient, isSecure: Boo *

Re: [PR] KAFKA-16539 Fix IncrementalAlterConfigs during ZK migration [kafka]

2024-04-19 Thread via GitHub
cmccabe commented on code in PR #15744: URL: https://github.com/apache/kafka/pull/15744#discussion_r1572716567 ## core/src/main/scala/kafka/zk/KafkaZkClient.scala: ## @@ -467,13 +470,33 @@ class KafkaZkClient private[zk] (zooKeeperClient: ZooKeeperClient, isSecure: Boo *

Re: [PR] KAFKA-16539 Fix IncrementalAlterConfigs during ZK migration [kafka]

2024-04-19 Thread via GitHub
cmccabe commented on code in PR #15744: URL: https://github.com/apache/kafka/pull/15744#discussion_r1572716567 ## core/src/main/scala/kafka/zk/KafkaZkClient.scala: ## @@ -467,13 +470,33 @@ class KafkaZkClient private[zk] (zooKeeperClient: ZooKeeperClient, isSecure: Boo *

[jira] [Updated] (KAFKA-16588) broker shutdown hangs when `log.segment.delete.delay.ms` is zero

2024-04-19 Thread Chia-Ping Tsai (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16588?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Chia-Ping Tsai updated KAFKA-16588: --- Description: see

[jira] [Updated] (KAFKA-16588) broker shutdown hangs when `log.segment.delete.delay.ms` is zero

2024-04-19 Thread Chia-Ping Tsai (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16588?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Chia-Ping Tsai updated KAFKA-16588: --- Description: see

[jira] [Created] (KAFKA-16588) broker shutdown hangs when `log.segment.delete.delay.ms` is zero

2024-04-19 Thread Chia-Ping Tsai (Jira)
Chia-Ping Tsai created KAFKA-16588: -- Summary: broker shutdown hangs when `log.segment.delete.delay.ms` is zero Key: KAFKA-16588 URL: https://issues.apache.org/jira/browse/KAFKA-16588 Project: Kafka

Re: [PR] KAFKA-16557: Fix toString of OffsetFetchRequestState [kafka]

2024-04-19 Thread via GitHub
phooq commented on PR #15750: URL: https://github.com/apache/kafka/pull/15750#issuecomment-2066909600 Thanks for the feedback @lianetm ! I agree with the point about the back-and-forth jumping between the child and the parent coming with this implementation, however asking each child

[jira] [Resolved] (KAFKA-16579) Revert changes to consumer_rolling_upgrade_test.py for the new async Consumer

2024-04-19 Thread Philip Nee (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16579?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Philip Nee resolved KAFKA-16579. Resolution: Fixed > Revert changes to consumer_rolling_upgrade_test.py for the new async Consumer

Re: [PR] MINOR: example.com moved [kafka]

2024-04-19 Thread via GitHub
akatona84 commented on PR #15758: URL: https://github.com/apache/kafka/pull/15758#issuecomment-2066839773 ipv6 alternates between 2606:2800:21f:cb07:6820:80da:af6b:8b2c and the old one, it's hard to fix the test like this. -- This is an automated message from the Apache Git Service.

Re: [PR] KAFKA-16579: Revert Consumer Rolling Upgrade [kafka]

2024-04-19 Thread via GitHub
lianetm commented on code in PR #15753: URL: https://github.com/apache/kafka/pull/15753#discussion_r1572569170 ## tests/kafkatest/tests/client/consumer_rolling_upgrade_test.py: ## @@ -56,12 +56,7 @@ def _verify_roundrobin_assignment(self, consumer):

Re: [PR] KAFKA-16579: Revert Consumer Rolling Upgrade [kafka]

2024-04-19 Thread via GitHub
lianetm commented on code in PR #15753: URL: https://github.com/apache/kafka/pull/15753#discussion_r1572569170 ## tests/kafkatest/tests/client/consumer_rolling_upgrade_test.py: ## @@ -56,12 +56,7 @@ def _verify_roundrobin_assignment(self, consumer):

Re: [PR] KAFKA-16563: retry pollEvent in KRaftMigrationDriver for retriable errors [kafka]

2024-04-19 Thread via GitHub
soarez commented on code in PR #15732: URL: https://github.com/apache/kafka/pull/15732#discussion_r1572557807 ## metadata/src/main/java/org/apache/kafka/metadata/migration/KRaftMigrationDriver.java: ## @@ -786,12 +773,29 @@ public void run() throws Exception { } }

Re: [PR] KAFKA-16563: retry pollEvent in KRaftMigrationDriver for retriable errors [kafka]

2024-04-19 Thread via GitHub
soarez commented on code in PR #15732: URL: https://github.com/apache/kafka/pull/15732#discussion_r1572550218 ## metadata/src/main/java/org/apache/kafka/metadata/migration/KRaftMigrationDriver.java: ## @@ -786,12 +773,29 @@ public void run() throws Exception { } }

Re: [PR] KAFKA-16272: Adding new coordinator related changes for connect_distributed.py [kafka]

2024-04-19 Thread via GitHub
lucasbru merged PR #15594: URL: https://github.com/apache/kafka/pull/15594 -- 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-16579: Revert Consumer Rolling Upgrade [kafka]

2024-04-19 Thread via GitHub
lucasbru commented on code in PR #15753: URL: https://github.com/apache/kafka/pull/15753#discussion_r1572534438 ## tests/kafkatest/tests/client/consumer_rolling_upgrade_test.py: ## @@ -56,12 +56,7 @@ def _verify_roundrobin_assignment(self, consumer):

Re: [PR] KAFKA-14226: feat(connect:transform): Introduce FieldPath abstraction [kafka]

2024-04-19 Thread via GitHub
jeqo commented on code in PR #15379: URL: https://github.com/apache/kafka/pull/15379#discussion_r1572509302 ## connect/transforms/src/test/java/org/apache/kafka/connect/transforms/field/FieldPathNotationTest.java: ## @@ -0,0 +1,149 @@ +/* + * Licensed to the Apache Software

Re: [PR] KAFKA-16579: Revert Consumer Rolling Upgrade [kafka]

2024-04-19 Thread via GitHub
lianetm commented on code in PR #15753: URL: https://github.com/apache/kafka/pull/15753#discussion_r1572508072 ## tests/kafkatest/tests/client/consumer_rolling_upgrade_test.py: ## @@ -56,12 +56,7 @@ def _verify_roundrobin_assignment(self, consumer):

Re: [PR] KAFKA-16557: Fix toString of OffsetFetchRequestState [kafka]

2024-04-19 Thread via GitHub
lianetm commented on PR #15750: URL: https://github.com/apache/kafka/pull/15750#issuecomment-2066727111 Thanks for the info @kirktrue , I see you mentioned it was a "suggested pattern", and I won't hold a strong position here but still sharing how I see it (struggling to find a value in it

Re: [PR] KAFKA-16103: AsyncConsumer should await pending async commits on commitSync and close [kafka]

2024-04-19 Thread via GitHub
lucasbru commented on code in PR #15613: URL: https://github.com/apache/kafka/pull/15613#discussion_r1572360676 ## core/src/test/scala/integration/kafka/api/PlaintextConsumerCommitTest.scala: ## @@ -304,6 +304,64 @@ class PlaintextConsumerCommitTest extends

Re: [PR] KAFKA-14226: feat(connect:transform): Introduce FieldPath abstraction [kafka]

2024-04-19 Thread via GitHub
jeqo commented on code in PR #15379: URL: https://github.com/apache/kafka/pull/15379#discussion_r1572436483 ## connect/transforms/src/test/java/org/apache/kafka/connect/transforms/ValueToKeyTest.java: ## @@ -60,6 +62,30 @@ public void schemaless() {

Re: [PR] KAFKA-16568: JMH Benchmarks for Server Side Rebalances [kafka]

2024-04-19 Thread via GitHub
dajac commented on code in PR #15717: URL: https://github.com/apache/kafka/pull/15717#discussion_r1572314707 ## jmh-benchmarks/src/main/java/org/apache/kafka/jmh/assignor/ServerSideAssignorBenchmark.java: ## @@ -0,0 +1,266 @@ +/* + * Licensed to the Apache Software Foundation

Re: [PR] KAFKA-14569: Migrate Connect's integration test EmbeddedKafkaCluster from ZK to KRaft mode [kafka]

2024-04-19 Thread via GitHub
mdedetrich commented on PR #13375: URL: https://github.com/apache/kafka/pull/13375#issuecomment-2066602056 @yashmayya Are you still working on this to get it over the finish line or is it okay for me to take over? -- This is an automated message from the Apache Git Service. To respond to

Re: [PR] KAFKA-16566: Fix consumer static membership system test with new protocol [kafka]

2024-04-19 Thread via GitHub
lianetm commented on PR #15738: URL: https://github.com/apache/kafka/pull/15738#issuecomment-2066577956 Hey @lucasbru , answering your questions : the new behaviour of the static membership regarding a member that joins with dup group instance Id is documented in

Re: [PR] KAFKA-16568: JMH Benchmarks for Server Side Rebalances [kafka]

2024-04-19 Thread via GitHub
dajac commented on code in PR #15717: URL: https://github.com/apache/kafka/pull/15717#discussion_r1572307205 ## jmh-benchmarks/src/main/java/org/apache/kafka/jmh/assignor/ServerSideAssignorBenchmark.java: ## @@ -0,0 +1,252 @@ +/* + * Licensed to the Apache Software Foundation

Re: [PR] KAFKA-16103: AsyncConsumer should await pending async commits on commitSync and close [kafka]

2024-04-19 Thread via GitHub
cadonna commented on PR #15613: URL: https://github.com/apache/kafka/pull/15613#issuecomment-2066478507 > > @lucasbru Thanks for the PR! > > The unit tests you added fail in the build and also for me locally. > > Plus, I have a question regarding the integration tests. > >

Re: [PR] KAFKA-16103: AsyncConsumer should await pending async commits on commitSync and close [kafka]

2024-04-19 Thread via GitHub
cadonna commented on code in PR #15613: URL: https://github.com/apache/kafka/pull/15613#discussion_r1572290394 ## core/src/test/scala/integration/kafka/api/PlaintextConsumerCommitTest.scala: ## @@ -304,6 +304,64 @@ class PlaintextConsumerCommitTest extends AbstractConsumerTest

[jira] [Commented] (KAFKA-16585) No way to forward message from punctuation method in the FixedKeyProcessor

2024-04-19 Thread Stanislav Spiridonov (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16585?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17838979#comment-17838979 ] Stanislav Spiridonov commented on KAFKA-16585: -- The case is relatively simple. I have

Re: [PR] KAFKA-16539 Fix IncrementalAlterConfigs during ZK migration [kafka]

2024-04-19 Thread via GitHub
soarez commented on code in PR #15744: URL: https://github.com/apache/kafka/pull/15744#discussion_r1572295590 ## core/src/test/scala/integration/kafka/zk/ZkMigrationIntegrationTest.scala: ## @@ -950,16 +980,47 @@ class ZkMigrationIntegrationTest {

Re: [PR] KAFKA-16463 System test for reverting migration to ZK [kafka]

2024-04-19 Thread via GitHub
mumrah commented on PR #15754: URL: https://github.com/apache/kafka/pull/15754#issuecomment-2066454711 ``` SESSION REPORT (ALL TESTS) ducktape version: 0.11.4 session_id: 2024-04-18--017

Re: [PR] MINOR: example.com moved [kafka]

2024-04-19 Thread via GitHub
akatona84 commented on PR #15758: URL: https://github.com/apache/kafka/pull/15758#issuecomment-2066418336 now the ipv6 address is changed to 2606:2800:220:1:248:1893:25c8:1946... -- This is an automated message from the Apache Git Service. To respond to the message, please log on to

[PR] MINOR: example.com moved [kafka]

2024-04-19 Thread via GitHub
akatona84 opened a new pull request, #15758: URL: https://github.com/apache/kafka/pull/15758 ### Committer Checklist (excluded from commit message) - [ ] Verify design and implementation - [ ] Verify test coverage and CI build status - [ ] Verify documentation (including upgrade

Re: [PR] KAFKA-16103: AsyncConsumer should await pending async commits on commitSync and close [kafka]

2024-04-19 Thread via GitHub
lucasbru commented on PR #15613: URL: https://github.com/apache/kafka/pull/15613#issuecomment-2066392315 > @lucasbru Thanks for the PR! > > The unit tests you added fail in the build and also for me locally. > > Plus, I have a question regarding the integration tests.

Re: [PR] KAFKA-16572: allow defining number of disks per broker in ClusterTest [kafka]

2024-04-19 Thread via GitHub
FrankYang0529 commented on code in PR #15745: URL: https://github.com/apache/kafka/pull/15745#discussion_r1572189143 ## core/src/test/java/kafka/test/junit/ClusterTestExtensions.java: ## @@ -179,8 +186,8 @@ private void processClusterTest(ExtensionContext context, ClusterTest

Re: [PR] MINOR: Fix io-[wait-]ratio metrics description [kafka]

2024-04-19 Thread via GitHub
chia7712 commented on PR #15722: URL: https://github.com/apache/kafka/pull/15722#issuecomment-2066283854 > Effectively, such test would verify the behavior of deprecated method. What do you think? we should verify the non-deprecated metrics should have correct doc which is not

Re: [PR] KAFKA-16424: remove truncated logs after alter dir [kafka]

2024-04-19 Thread via GitHub
chia7712 commented on PR #15616: URL: https://github.com/apache/kafka/pull/15616#issuecomment-2066277923 This PR is good but it seems to me `LogSegment` should NOT guess the directory structure managed by upper class (i.e `LogManager`). It seems the root cause is caused by following

Re: [PR] KAFKA-16561: Disable `allow.auto.create.topics` in MirrorMaker2 Consumer Config [kafka]

2024-04-19 Thread via GitHub
OmniaGM commented on PR #15728: URL: https://github.com/apache/kafka/pull/15728#issuecomment-2066275194 Thanks for getting the KIP out there for discussion and for fixing the tests. Should this PR be a draft until we have the KIP voted in by the community? -- This is an automated

Re: [PR] KAFKA-16561: Disable `allow.auto.create.topics` in MirrorMaker2 Consumer Config [kafka]

2024-04-19 Thread via GitHub
aaron-ai commented on PR #15728: URL: https://github.com/apache/kafka/pull/15728#issuecomment-2066276867 > Thanks for getting the KIP out there for discussion and for fixing the tests. Should this PR be a draft until we have the KIP voted in by the community? OK -- This is an

Re: [PR] KAFKA-16561: Disable `allow.auto.create.topics` in MirrorMaker2 Consumer Config [kafka]

2024-04-19 Thread via GitHub
OmniaGM commented on code in PR #15728: URL: https://github.com/apache/kafka/pull/15728#discussion_r1572155918 ## connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorConnectorConfig.java: ## @@ -169,6 +170,7 @@ static Map sourceConsumerConfig(Map props) {

Re: [PR] KAFKA-16539 Fix IncrementalAlterConfigs during ZK migration [kafka]

2024-04-19 Thread via GitHub
showuon commented on code in PR #15744: URL: https://github.com/apache/kafka/pull/15744#discussion_r1572140166 ## core/src/main/scala/kafka/zk/KafkaZkClient.scala: ## @@ -467,13 +470,33 @@ class KafkaZkClient private[zk] (zooKeeperClient: ZooKeeperClient, isSecure: Boo *

Re: [PR] KAFKA-16572: allow defining number of disks per broker in ClusterTest [kafka]

2024-04-19 Thread via GitHub
chia7712 commented on code in PR #15745: URL: https://github.com/apache/kafka/pull/15745#discussion_r1572122943 ## core/src/test/java/kafka/test/junit/ClusterTestExtensions.java: ## @@ -179,8 +186,8 @@ private void processClusterTest(ExtensionContext context, ClusterTest

Re: [PR] KAFKA-16572: allow defining number of disks per broker in ClusterTest [kafka]

2024-04-19 Thread via GitHub
FrankYang0529 commented on code in PR #15745: URL: https://github.com/apache/kafka/pull/15745#discussion_r1572114481 ## core/src/test/java/kafka/test/junit/ClusterTestExtensions.java: ## @@ -179,8 +186,8 @@ private void processClusterTest(ExtensionContext context, ClusterTest

Re: [PR] KAFKA-16561: Disable `allow.auto.create.topics` in MirrorMaker2 Consumer Config [kafka]

2024-04-19 Thread via GitHub
aaron-ai commented on PR #15728: URL: https://github.com/apache/kafka/pull/15728#issuecomment-2066221145 KIP has been created here: https://cwiki.apache.org/confluence/display/KAFKA/KIP-1039%3A+Disable+automatic+topic+creation+for+MirrorMaker2+consumers -- This is an automated message

Re: [PR] KAFKA-16103: AsyncConsumer should await pending async commits on commitSync and close [kafka]

2024-04-19 Thread via GitHub
lucasbru commented on code in PR #15613: URL: https://github.com/apache/kafka/pull/15613#discussion_r1572109503 ## core/src/test/scala/integration/kafka/api/PlaintextConsumerCommitTest.scala: ## @@ -304,6 +304,64 @@ class PlaintextConsumerCommitTest extends

Re: [PR] KAFKA-16103: AsyncConsumer should await pending async commits on commitSync and close [kafka]

2024-04-19 Thread via GitHub
lucasbru commented on code in PR #15613: URL: https://github.com/apache/kafka/pull/15613#discussion_r1572109078 ## core/src/test/scala/integration/kafka/api/PlaintextConsumerCommitTest.scala: ## @@ -304,6 +304,64 @@ class PlaintextConsumerCommitTest extends

Re: [PR] KAFKA-16579: Revert Consumer Rolling Upgrade [kafka]

2024-04-19 Thread via GitHub
lucasbru merged PR #15753: URL: https://github.com/apache/kafka/pull/15753 -- 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-16566: Fix consumer static membership system test with new protocol [kafka]

2024-04-19 Thread via GitHub
lucasbru commented on PR #15738: URL: https://github.com/apache/kafka/pull/15738#issuecomment-2066195894 LGTM, thanks! Merging this, however: * Have we discussed the behavioral difference with broker team / David? * Have we documented the behavioral difference anywhere? --

Re: [PR] KAFKA-16566: Fix consumer static membership system test with new protocol [kafka]

2024-04-19 Thread via GitHub
lucasbru merged PR #15738: URL: https://github.com/apache/kafka/pull/15738 -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail:

[PR] MINOR:fix hint in fetchOffsetForTimestamp [kafka]

2024-04-19 Thread via GitHub
hudeqi opened a new pull request, #15757: URL: https://github.com/apache/kafka/pull/15757 A clear hint meaning error: The actual logic is that an error is thrown only when the high watermark lags behind the epoch start offset, but the hint meaning is opposite. -- This is an automated

[jira] [Commented] (KAFKA-15709) KRaft support in ServerStartupTest

2024-04-19 Thread Zihao Lin (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15709?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17838923#comment-17838923 ] Zihao Lin commented on KAFKA-15709: --- [~mdedetrich] feel free to take over > KRaft support in

Re: [PR] KAFKA-15265: Add Remote Log Manager quota manager [kafka]

2024-04-19 Thread via GitHub
funky-eyes commented on code in PR #15625: URL: https://github.com/apache/kafka/pull/15625#discussion_r1572073595 ## core/src/main/java/kafka/log/remote/quota/RLMQuotaManager.java: ## Review Comment: I have a question, in fact, what this PR does is to provide a standard

Re: [PR] KAFKA-16572: allow defining number of disks per broker in ClusterTest [kafka]

2024-04-19 Thread via GitHub
chia7712 commented on code in PR #15745: URL: https://github.com/apache/kafka/pull/15745#discussion_r1572064864 ## core/src/test/java/kafka/test/junit/ClusterTestExtensions.java: ## @@ -179,8 +186,8 @@ private void processClusterTest(ExtensionContext context, ClusterTest

Re: [PR] KAFKA-16552: Create an internal config to control InitialTaskDelayMs in LogManager to speed up tests [kafka]

2024-04-19 Thread via GitHub
chia7712 commented on PR #15719: URL: https://github.com/apache/kafka/pull/15719#issuecomment-2066133702 @brandboat this PR is great. However, I'd like to merge it after #15569. #15569 is a huge PR which refactor the `KafkaConfig` and `LogConfig`, and I try to alleviate the pain of fixing

Re: [PR] KAFKA-16552: Create an internal config to control InitialTaskDelayMs in LogManager to speed up tests [kafka]

2024-04-19 Thread via GitHub
brandboat commented on PR #15719: URL: https://github.com/apache/kafka/pull/15719#issuecomment-2066136873 > @brandboat this PR is great. However, I'd like to merge it after https://github.com/apache/kafka/pull/15569. https://github.com/apache/kafka/pull/15569 is a huge PR which refactor

Re: [PR] KAFKA-16572: allow defining number of disks per broker in ClusterTest [kafka]

2024-04-19 Thread via GitHub
FrankYang0529 commented on PR #15745: URL: https://github.com/apache/kafka/pull/15745#issuecomment-2066132584 Hi @gaurav-narula and @chia7712, I have addressed all comments. Thanks for your review. -- This is an automated message from the Apache Git Service. To respond to the message,

Re: [PR] KAFKA-16424: remove truncated logs after alter dir [kafka]

2024-04-19 Thread via GitHub
showuon commented on PR #15616: URL: https://github.com/apache/kafka/pull/15616#issuecomment-2066130114 @johnnychhsu , do you have any other comments? I'll merge this at the weekend if no other comments. Thanks. -- This is an automated message from the Apache Git Service. To respond to

Re: [PR] KAFKA-15853: Move KafkaConfig log properties and docs out of core [kafka]

2024-04-19 Thread via GitHub
chia7712 commented on PR #15569: URL: https://github.com/apache/kafka/pull/15569#issuecomment-2066120621 ``` [2024-04-18T17:26:16.149Z] [ant:checkstyle] [ERROR]

Re: [PR] KAFKA-16563: retry pollEvent in KRaftMigrationDriver for retriable errors [kafka]

2024-04-19 Thread via GitHub
showuon commented on code in PR #15732: URL: https://github.com/apache/kafka/pull/15732#discussion_r1572018203 ## metadata/src/test/java/org/apache/kafka/metadata/migration/KRaftMigrationDriverTest.java: ## @@ -881,10 +937,18 @@ public List> recordBatches() {

Re: [PR] KAFKA-16563: retry pollEvent in KRaftMigrationDriver for retriable errors [kafka]

2024-04-19 Thread via GitHub
showuon commented on code in PR #15732: URL: https://github.com/apache/kafka/pull/15732#discussion_r1572018955 ## metadata/src/test/java/org/apache/kafka/metadata/migration/KRaftMigrationDriverTest.java: ## @@ -881,10 +937,18 @@ public List> recordBatches() {

Re: [PR] KAFKA-15838: ExtractField and InsertField NULL Values are replaced by default value even in NULLABLE fields [kafka]

2024-04-19 Thread via GitHub
mfvitale commented on PR #15756: URL: https://github.com/apache/kafka/pull/15756#issuecomment-2066045097 > Hi @mfvitale, thanks for the PR! This is adding new configurations to transformations so this will require a

Re: [PR] KAFKA-15838: ExtractField and InsertField NULL Values are replaced by default value even in NULLABLE fields [kafka]

2024-04-19 Thread via GitHub
mimaison commented on PR #15756: URL: https://github.com/apache/kafka/pull/15756#issuecomment-2066023794 Hi @mfvitale, thanks for the PR! This is adding new configurations to transformations so this will require a

Re: [PR] KAFKA-16563: retry pollEvent in KRaftMigrationDriver for retriable errors [kafka]

2024-04-19 Thread via GitHub
showuon commented on code in PR #15732: URL: https://github.com/apache/kafka/pull/15732#discussion_r1571981454 ## metadata/src/main/java/org/apache/kafka/metadata/migration/KRaftMigrationDriver.java: ## @@ -786,12 +773,29 @@ public void run() throws Exception { }

Re: [PR] KAFKA-16563: retry pollEvent in KRaftMigrationDriver for retriable errors [kafka]

2024-04-19 Thread via GitHub
showuon commented on code in PR #15732: URL: https://github.com/apache/kafka/pull/15732#discussion_r1571963909 ## metadata/src/main/java/org/apache/kafka/metadata/migration/KRaftMigrationDriver.java: ## @@ -786,12 +773,29 @@ public void run() throws Exception { }

Re: [PR] KAFKA-16483: migrate DeleteOffsetsConsumerGroupCommandIntegrationTest to use ClusterTestExtensions [kafka]

2024-04-19 Thread via GitHub
chia7712 commented on PR #15679: URL: https://github.com/apache/kafka/pull/15679#issuecomment-2065947030 > Do you think that we should revert unstable.api.versions.enable change and try again? Thanks. Yep -- This is an automated message from the Apache Git Service. To respond to

Re: [PR] MINOR: Use Parametrized types correctly in RemoteLogMetadataSerde [kafka]

2024-04-19 Thread via GitHub
jlprat merged PR #13824: URL: https://github.com/apache/kafka/pull/13824 -- 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-15963) Flaky test: testBrokerHeartbeatDuringMigration [3] 3.6-IV0 – org.apache.kafka.controller.QuorumControllerTest

2024-04-19 Thread Josep Prat (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15963?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17838883#comment-17838883 ] Josep Prat commented on KAFKA-15963: it failed again but 3.6-IV1 this time:

Re: [PR] MINOR: Use Parametrized types correctly in RemoteLogMetadataSerde [kafka]

2024-04-19 Thread via GitHub
jlprat commented on PR #13824: URL: https://github.com/apache/kafka/pull/13824#issuecomment-2065886831 All tests failing for this build were known flaky tests. Merging -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use

  1   2   >