Re: [PR] KAFKA-15774: introduce internal StoreFactory [kafka]

2023-11-07 Thread via GitHub
mjsax commented on code in PR #14659: URL: https://github.com/apache/kafka/pull/14659#discussion_r1386110614 ## streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilder.java: ## @@ -16,30 +16,9 @@ */ package

Re: [PR] KAFKA-15022: add config for balance subtopology in rack aware task assignment [kafka]

2023-11-07 Thread via GitHub
ex172000 commented on code in PR #14711: URL: https://github.com/apache/kafka/pull/14711#discussion_r1386109294 ## streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java: ## @@ -760,13 +760,14 @@ public class StreamsConfig extends AbstractConfig { public

[jira] [Commented] (KAFKA-14419) Failed SyncGroup leading to partitions lost due to processing during rebalances

2023-11-07 Thread Matthias J. Sax (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-14419?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17783897#comment-17783897 ] Matthias J. Sax commented on KAFKA-14419: - [~Carlstedt] – did not look into too many details,

[jira] [Commented] (KAFKA-13627) Topology changes shouldn't require a full reset of local state

2023-11-07 Thread Matthias J. Sax (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-13627?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17783894#comment-17783894 ] Matthias J. Sax commented on KAFKA-13627: - As you pointed out, the KIP did not make progress for

[jira] [Updated] (KAFKA-15797) Flaky test EosV2UpgradeIntegrationTest.shouldUpgradeFromEosAlphaToEosV2[true]

2023-11-07 Thread Matthias J. Sax (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15797?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax updated KAFKA-15797: Component/s: streams unit tests > Flaky test

[jira] [Updated] (KAFKA-15798) Flaky Test NamedTopologyIntegrationTest.shouldAddAndRemoveNamedTopologiesBeforeStartingAndRouteQueriesToCorrectTopology()

2023-11-07 Thread Matthias J. Sax (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15798?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax updated KAFKA-15798: Component/s: streams unit tests > Flaky Test >

[jira] [Commented] (KAFKA-3700) CRL support

2023-11-07 Thread Igor Shipenkov (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-3700?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17783882#comment-17783882 ] Igor Shipenkov commented on KAFKA-3700: --- For people, looking for OCSP support. Use Oracle Java

Re: [PR] KAFKA-14683 Migrate #testStartPaused to Mockito [kafka]

2023-11-07 Thread via GitHub
hgeraldino commented on PR #14663: URL: https://github.com/apache/kafka/pull/14663#issuecomment-1800971956 One thing I'd like to know is if this approach of multiple bite-sized PRs seems reasonable or not. Future PRs might include refactor of more than one method, but I'll try my

Re: [PR] KAFKA-15324 Do not bump leader epoch when changing the replica set [kafka]

2023-11-07 Thread via GitHub
github-actions[bot] commented on PR #14170: URL: https://github.com/apache/kafka/pull/14170#issuecomment-1800956612 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-15774: refactor windowed stores to use StoreFactory [kafka]

2023-11-07 Thread via GitHub
ableegoldman commented on code in PR #14708: URL: https://github.com/apache/kafka/pull/14708#discussion_r1385859246 ## streams/src/main/java/org/apache/kafka/streams/kstream/internals/SessionWindowStoreMaterializer.java: ## @@ -0,0 +1,136 @@ +/* + * Licensed to the Apache

[PR] KAFKA-15022: introduce interface to control graph constructor [kafka]

2023-11-07 Thread via GitHub
lihaosky opened a new pull request, #14714: URL: https://github.com/apache/kafka/pull/14714 ### Description Refactor graph construction and assignment in `RackAwareAssignor` to new interface. Will add implementation for subtopology case and unit test later -- This is an automated

Re: [PR] KAFKA-15681: Add support of client-metrics in kafka-configs.sh (KIP-714) [kafka]

2023-11-07 Thread via GitHub
junrao commented on code in PR #14632: URL: https://github.com/apache/kafka/pull/14632#discussion_r1385792007 ## core/src/main/scala/kafka/server/DynamicConfig.scala: ## @@ -111,6 +112,16 @@ object DynamicConfig { } } + object ClientMetrics { +private val

Re: [PR] MINOR: Rewrite the meta.properties handling code in Java and fix some issues [kafka]

2023-11-07 Thread via GitHub
pprovenzano commented on code in PR #14628: URL: https://github.com/apache/kafka/pull/14628#discussion_r1385811376 ## core/src/main/scala/kafka/server/KafkaServer.scala: ## @@ -270,10 +271,31 @@ class KafkaServer( logDirFailureChannel = new

Re: [PR] MINOR: Rewrite the meta.properties handling code in Java and fix some issues [kafka]

2023-11-07 Thread via GitHub
pprovenzano commented on code in PR #14628: URL: https://github.com/apache/kafka/pull/14628#discussion_r1385811376 ## core/src/main/scala/kafka/server/KafkaServer.scala: ## @@ -270,10 +271,31 @@ class KafkaServer( logDirFailureChannel = new

Re: [PR] MINOR: Rewrite the meta.properties handling code in Java and fix some issues [kafka]

2023-11-07 Thread via GitHub
pprovenzano commented on code in PR #14628: URL: https://github.com/apache/kafka/pull/14628#discussion_r1385810018 ## core/src/main/scala/kafka/server/KafkaServer.scala: ## @@ -270,10 +271,31 @@ class KafkaServer( logDirFailureChannel = new

Re: [PR] MINOR: Rewrite the meta.properties handling code in Java and fix some issues [kafka]

2023-11-07 Thread via GitHub
pprovenzano commented on code in PR #14628: URL: https://github.com/apache/kafka/pull/14628#discussion_r1385810018 ## core/src/main/scala/kafka/server/KafkaServer.scala: ## @@ -270,10 +271,31 @@ class KafkaServer( logDirFailureChannel = new

Re: [PR] MINOR: Rewrite the meta.properties handling code in Java and fix some issues [kafka]

2023-11-07 Thread via GitHub
pprovenzano commented on code in PR #14628: URL: https://github.com/apache/kafka/pull/14628#discussion_r1385805130 ## metadata/src/main/java/org/apache/kafka/metadata/properties/MetaPropertiesEnsemble.java: ## @@ -0,0 +1,575 @@ +/* + * Licensed to the Apache Software Foundation

Re: [PR] KAFKA-15174: Ensure CommitAsync propagate the exception to the user [kafka]

2023-11-07 Thread via GitHub
kirktrue commented on code in PR #14680: URL: https://github.com/apache/kafka/pull/14680#discussion_r1380596565 ## clients/src/main/java/org/apache/kafka/clients/consumer/internals/PrototypeAsyncConsumer.java: ## @@ -381,20 +394,33 @@ public void

Re: [PR] MINOR: Rewrite the meta.properties handling code in Java and fix some issues [kafka]

2023-11-07 Thread via GitHub
mumrah commented on code in PR #14628: URL: https://github.com/apache/kafka/pull/14628#discussion_r1385786809 ## metadata/src/main/java/org/apache/kafka/metadata/properties/MetaPropertiesVersion.java: ## @@ -0,0 +1,71 @@ +/* + * Licensed to the Apache Software Foundation (ASF)

Re: [PR] KAFKA-15780: Wait for consistent KRaft metadata when creating or deleting topics [kafka]

2023-11-07 Thread via GitHub
jolshan commented on PR #14713: URL: https://github.com/apache/kafka/pull/14713#issuecomment-1800705043 @divijvaidya This one was missing the change in some test files, so we should probably review TopicCommandIntegrationTest. I suspect 3.5 will have more conflicts. -- This is an

[PR] KAFKA-15780: Wait for consistent KRaft metadata when creating or deleting topics [kafka]

2023-11-07 Thread via GitHub
jolshan opened a new pull request, #14713: URL: https://github.com/apache/kafka/pull/14713 TestUtils.createTopicWithAdmin calls waitForAllPartitionsMetadata which waits for partition(s) to be present in each brokers' metadata cache. This is a sufficient check in ZK mode because the

[PR] KAFKA-15653: Pass requestLocal as argument to callback so we use the correct one for the thread [kafka]

2023-11-07 Thread via GitHub
jolshan opened a new pull request, #14712: URL: https://github.com/apache/kafka/pull/14712 With the new callback mechanism we were accidentally passing context with the wrong request local. Now include a RequestLocal as an explicit argument to the callback. Also make the arguments

Re: [PR] KAFKA-15653: Pass requestLocal as argument to callback so we use the correct one for the thread [kafka]

2023-11-07 Thread via GitHub
jolshan merged PR #14629: URL: https://github.com/apache/kafka/pull/14629 -- 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-15653: Pass requestLocal as argument to callback so we use the correct one for the thread [kafka]

2023-11-07 Thread via GitHub
jolshan commented on PR #14629: URL: https://github.com/apache/kafka/pull/14629#issuecomment-1800331351 Thanks @divijvaidya. I was taking a look at Gradle enterprise as well. But the link is helpful -- This is an automated message from the Apache Git Service. To respond to the message,

Re: [PR] KAFKA-15277: Design & implement support for internal Consumer delegates [kafka]

2023-11-07 Thread via GitHub
kirktrue commented on code in PR #14670: URL: https://github.com/apache/kafka/pull/14670#discussion_r1385683761 ## clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java: ## @@ -721,16 +802,17 @@ public void

Re: [PR] KAFKA-15277: Design & implement support for internal Consumer delegates [kafka]

2023-11-07 Thread via GitHub
kirktrue commented on code in PR #14670: URL: https://github.com/apache/kafka/pull/14670#discussion_r1385682384 ## clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java: ## @@ -222,42 +214,74 @@ public void cleanup() { } } -@Test -

Re: [PR] KAFKA-15277: Design & implement support for internal Consumer delegates [kafka]

2023-11-07 Thread via GitHub
kirktrue commented on code in PR #14670: URL: https://github.com/apache/kafka/pull/14670#discussion_r1385680899 ## clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerUtils.java: ## @@ -211,4 +226,17 @@ else if (t instanceof KafkaException)

Re: [PR] KAFKA-15277: Design & implement support for internal Consumer delegates [kafka]

2023-11-07 Thread via GitHub
kirktrue commented on PR #14670: URL: https://github.com/apache/kafka/pull/14670#issuecomment-1800305244 `KafkaConsumerTest` is a test suite that exercised the functionality of the "existing" `KafkaConsumer` with 100 tests. With this change, the suite was refactored to allow each of those

[jira] [Created] (KAFKA-15798) Flaky Test NamedTopologyIntegrationTest.shouldAddAndRemoveNamedTopologiesBeforeStartingAndRouteQueriesToCorrectTopology()

2023-11-07 Thread Justine Olshan (Jira)
Justine Olshan created KAFKA-15798: -- Summary: Flaky Test NamedTopologyIntegrationTest.shouldAddAndRemoveNamedTopologiesBeforeStartingAndRouteQueriesToCorrectTopology() Key: KAFKA-15798 URL:

[jira] [Created] (KAFKA-15797) Flaky test EosV2UpgradeIntegrationTest.shouldUpgradeFromEosAlphaToEosV2[true]

2023-11-07 Thread Justine Olshan (Jira)
Justine Olshan created KAFKA-15797: -- Summary: Flaky test EosV2UpgradeIntegrationTest.shouldUpgradeFromEosAlphaToEosV2[true] Key: KAFKA-15797 URL: https://issues.apache.org/jira/browse/KAFKA-15797

Re: [PR] KAFKA-15277: Design & implement support for internal Consumer delegates [kafka]

2023-11-07 Thread via GitHub
kirktrue commented on code in PR #14670: URL: https://github.com/apache/kafka/pull/14670#discussion_r1385631897 ## clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerDelegate.java: ## @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation

Re: [PR] KAFKA-15277: Design & implement support for internal Consumer delegates [kafka]

2023-11-07 Thread via GitHub
kirktrue commented on code in PR #14670: URL: https://github.com/apache/kafka/pull/14670#discussion_r1385631654 ## clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java: ## @@ -222,42 +214,74 @@ public void cleanup() { } } -@Test -

[jira] [Commented] (KAFKA-15690) EosIntegrationTest is flaky.

2023-11-07 Thread Justine Olshan (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15690?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17783810#comment-17783810 ] Justine Olshan commented on KAFKA-15690: I saw this as well on my build

[jira] [Updated] (KAFKA-15690) EosIntegrationTest is flaky.

2023-11-07 Thread Justine Olshan (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15690?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Justine Olshan updated KAFKA-15690: --- Labels: flaky-test (was: ) > EosIntegrationTest is flaky. > >

[jira] [Commented] (KAFKA-15699) MirrorConnectorsIntegrationBaseTest is flaky

2023-11-07 Thread Justine Olshan (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15699?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17783807#comment-17783807 ] Justine Olshan commented on KAFKA-15699: I've seen this on my builds as well:

[jira] [Comment Edited] (KAFKA-15760) org.apache.kafka.trogdor.coordinator.CoordinatorTest.testTaskRequestWithOldStartMsGetsUpdated is flaky

2023-11-07 Thread Justine Olshan (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15760?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17783799#comment-17783799 ] Justine Olshan edited comment on KAFKA-15760 at 11/7/23 9:46 PM: - I also

[jira] [Comment Edited] (KAFKA-15759) DescribeClusterRequestTest is flaky

2023-11-07 Thread Justine Olshan (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15759?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17783798#comment-17783798 ] Justine Olshan edited comment on KAFKA-15759 at 11/7/23 9:46 PM: - Think

[jira] [Comment Edited] (KAFKA-15700) FetchFromFollowerIntegrationTest is flaky

2023-11-07 Thread Justine Olshan (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15700?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17783800#comment-17783800 ] Justine Olshan edited comment on KAFKA-15700 at 11/7/23 9:46 PM: - Also

[jira] [Commented] (KAFKA-15772) Flaky test TransactionsWithTieredStoreTest

2023-11-07 Thread Justine Olshan (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15772?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17783804#comment-17783804 ] Justine Olshan commented on KAFKA-15772: I have seen this one be a bit flaky as well.

Re: [PR] KAFKA-15277: Design & implement support for internal Consumer delegates [kafka]

2023-11-07 Thread via GitHub
kirktrue commented on code in PR #14670: URL: https://github.com/apache/kafka/pull/14670#discussion_r1385620881 ## clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java: ## @@ -222,42 +214,74 @@ public void cleanup() { } } -@Test -

Re: [PR] KAFKA-15277: Design & implement support for internal Consumer delegates [kafka]

2023-11-07 Thread via GitHub
kirktrue commented on code in PR #14670: URL: https://github.com/apache/kafka/pull/14670#discussion_r1385620217 ## clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThread.java: ## @@ -247,12 +247,10 @@ private void closeInternal(final Duration

Re: [PR] KAFKA-15653: Pass requestLocal as argument to callback so we use the correct one for the thread [kafka]

2023-11-07 Thread via GitHub
divijvaidya commented on PR #14629: URL: https://github.com/apache/kafka/pull/14629#issuecomment-1800218167 Hey @jolshan - You might already know this but sharing in case you don't. A tool I use to quickly find whether a test is flaky is

Re: [PR] KAFKA-15277: Design & implement support for internal Consumer delegates [kafka]

2023-11-07 Thread via GitHub
kirktrue commented on code in PR #14670: URL: https://github.com/apache/kafka/pull/14670#discussion_r1385617039 ## clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerDelegateCreator.java: ## @@ -0,0 +1,114 @@ +/* + * Licensed to the Apache Software

[jira] [Commented] (KAFKA-15700) FetchFromFollowerIntegrationTest is flaky

2023-11-07 Thread Justine Olshan (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15700?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17783800#comment-17783800 ] Justine Olshan commented on KAFKA-15700: Also saw this one :)  KAFKA-15020 and KAFKA-15105 may

[jira] [Commented] (KAFKA-15760) org.apache.kafka.trogdor.coordinator.CoordinatorTest.testTaskRequestWithOldStartMsGetsUpdated is flaky

2023-11-07 Thread Justine Olshan (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15760?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17783799#comment-17783799 ] Justine Olshan commented on KAFKA-15760: I also saw this failing today. >

[jira] [Commented] (KAFKA-15759) DescribeClusterRequestTest is flaky

2023-11-07 Thread Justine Olshan (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15759?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17783798#comment-17783798 ] Justine Olshan commented on KAFKA-15759: Think this might be a duplicate of KAFKA-15419.

Re: [PR] KAFKA-15277: Design & implement support for internal Consumer delegates [kafka]

2023-11-07 Thread via GitHub
kirktrue commented on code in PR #14670: URL: https://github.com/apache/kafka/pull/14670#discussion_r1385615790 ## clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerDelegate.java: ## @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation

Re: [PR] KAFKA-15277: Design & implement support for internal Consumer delegates [kafka]

2023-11-07 Thread via GitHub
kirktrue commented on code in PR #14670: URL: https://github.com/apache/kafka/pull/14670#discussion_r1385614421 ## clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerDelegate.java: ## @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation

[PR] KAFKA-15022: add config for balance subtopology in rack aware task assignment [kafka]

2023-11-07 Thread via GitHub
lihaosky opened a new pull request, #14711: URL: https://github.com/apache/kafka/pull/14711 Adding config for rack aware assignment strategy which will compute assignment with min cost and balance tasks of same subtopology to different clients ### Committer Checklist (excluded from

Re: [PR] KAFKA-15277: Design & implement support for internal Consumer delegates [kafka]

2023-11-07 Thread via GitHub
kirktrue commented on code in PR #14670: URL: https://github.com/apache/kafka/pull/14670#discussion_r1385612995 ## clients/src/main/java/org/apache/kafka/clients/consumer/internals/LegacyKafkaConsumer.java: ## @@ -0,0 +1,2554 @@ +/* + * Licensed to the Apache Software

Re: [PR] MINOR: Rewrite the meta.properties handling code in Java and fix some issues [kafka]

2023-11-07 Thread via GitHub
cmccabe commented on PR #14628: URL: https://github.com/apache/kafka/pull/14628#issuecomment-1800111262 > This PR does not work for ZK. You cannot create the directories in LogManager.scala but then write the directoryIds later. The directoryIds must be generated either when LogManager

Re: [PR] KAFKA-15653: Pass requestLocal as argument to callback so we use the correct one for the thread [kafka]

2023-11-07 Thread via GitHub
junrao commented on PR #14629: URL: https://github.com/apache/kafka/pull/14629#issuecomment-1800110570 @jolshan : Thanks for the analysis. If the test failures are unrelated, feel free to merge the PR. It would be useful to file jiras to track new transient test failures. -- This is an

Re: [PR] KAFKA-15778 & KAFKA-15779: Implement metrics manager (KIP-714) [kafka]

2023-11-07 Thread via GitHub
apoorvmittal10 commented on PR #14699: URL: https://github.com/apache/kafka/pull/14699#issuecomment-1800097217 @junrao @hachikuji @AndrewJSchofield @mjsax Please if I can get feedback on the PR. -- This is an automated message from the Apache Git Service. To respond to the message,

Re: [PR] KAFKA-15681: Add support of client-metrics in kafka-configs.sh (KIP-714) [kafka]

2023-11-07 Thread via GitHub
apoorvmittal10 commented on PR #14632: URL: https://github.com/apache/kafka/pull/14632#issuecomment-1800093637 @junrao @hachikuji @AndrewJSchofield @mjsax Please if I can get the feedback on the PR. -- This is an automated message from the Apache Git Service. To respond to the message,

[PR] KAFKA-15327: ensure the commit manager commit on close [kafka]

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

Re: [PR] KAFKA-15661: KIP-951: Server side changes [kafka]

2023-11-07 Thread via GitHub
chb2ab commented on code in PR #1: URL: https://github.com/apache/kafka/pull/1#discussion_r1385525549 ## core/src/main/scala/kafka/server/KafkaApis.scala: ## @@ -562,6 +562,26 @@ class KafkaApis(val requestChannel: RequestChannel, } } + case class

Re: [PR] PROPOSAL: support async event execution in group coordinator [kafka]

2023-11-07 Thread via GitHub
artemlivshits commented on code in PR #14705: URL: https://github.com/apache/kafka/pull/14705#discussion_r1385493045 ## group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/CoordinatorRuntime.java: ## @@ -659,9 +660,21 @@ public TopicPartition key() {

Re: [PR] KAFKA-15360: Include dirs in BrokerRegistration [kafka]

2023-11-07 Thread via GitHub
pprovenzano commented on PR #14392: URL: https://github.com/apache/kafka/pull/14392#issuecomment-1799807037 So, I think here we just want to set the list of directory UUIDs that we know about and assume they are online. We can later set a known directory to offline with our first

[jira] [Commented] (KAFKA-15546) Transactions tool duration field confusing for completed transactions

2023-11-07 Thread Raman Verma (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15546?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17783769#comment-17783769 ] Raman Verma commented on KAFKA-15546: - [~hifly81] I have opened this KIP for changes to

Re: [PR] KAFKA-15661: KIP-951: Server side changes [kafka]

2023-11-07 Thread via GitHub
jolshan commented on code in PR #1: URL: https://github.com/apache/kafka/pull/1#discussion_r1385453402 ## core/src/main/scala/kafka/server/KafkaApis.scala: ## @@ -562,6 +562,26 @@ class KafkaApis(val requestChannel: RequestChannel, } } + case class

Re: [PR] KAFKA-15470: Allow creating connectors in a stopped state [kafka]

2023-11-07 Thread via GitHub
C0urante commented on code in PR #14704: URL: https://github.com/apache/kafka/pull/14704#discussion_r1385398359 ## connect/runtime/src/main/java/org/apache/kafka/connect/cli/ConnectStandalone.java: ## @@ -87,6 +98,61 @@ protected void processExtraArgs(Herder herder, Connect

[jira] [Commented] (KAFKA-13627) Topology changes shouldn't require a full reset of local state

2023-11-07 Thread Ben Ellis (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-13627?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17783750#comment-17783750 ] Ben Ellis commented on KAFKA-13627: --- Discussion on KIP-816 seems to have gone idle since 2022. Is that

Re: [PR] KAFKA-15661: KIP-951: Server side changes [kafka]

2023-11-07 Thread via GitHub
chb2ab commented on code in PR #1: URL: https://github.com/apache/kafka/pull/1#discussion_r1385430343 ## core/src/main/scala/kafka/server/KafkaApis.scala: ## @@ -562,6 +562,26 @@ class KafkaApis(val requestChannel: RequestChannel, } } + case class

Re: [PR] KAFKA-15653: Pass requestLocal as argument to callback so we use the correct one for the thread [kafka]

2023-11-07 Thread via GitHub
jolshan commented on PR #14629: URL: https://github.com/apache/kafka/pull/14629#issuecomment-1799582958 I took a look at the most recent 19 failures. The majority of them are failing on trunk in the last few (5 or so) runs. I will take a look at the remaining ones I didn't see failing

Re: [PR] PROPOSAL: support async event execution in group coordinator [kafka]

2023-11-07 Thread via GitHub
junrao commented on code in PR #14705: URL: https://github.com/apache/kafka/pull/14705#discussion_r1385389618 ## group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/CoordinatorRuntime.java: ## @@ -659,9 +660,21 @@ public TopicPartition key() {

Re: [PR] KAFKA-15357: Aggregate and propagate assignments [kafka]

2023-11-07 Thread via GitHub
pprovenzano commented on code in PR #14369: URL: https://github.com/apache/kafka/pull/14369#discussion_r1385399770 ## core/src/main/java/kafka/server/AssignmentsManager.java: ## @@ -0,0 +1,386 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + *

Re: [PR] KAFKA-15780: Wait for consistent KRaft metadata when creating or deleting topics [kafka]

2023-11-07 Thread via GitHub
jolshan commented on PR #14695: URL: https://github.com/apache/kafka/pull/14695#issuecomment-1799528544 I can do that if we think it is useful. -- 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-15357: Aggregate and propagate assignments [kafka]

2023-11-07 Thread via GitHub
pprovenzano commented on code in PR #14369: URL: https://github.com/apache/kafka/pull/14369#discussion_r1385385403 ## core/src/main/scala/kafka/server/ReplicaManager.scala: ## @@ -2296,13 +2298,41 @@ class ReplicaManager(val config: KafkaConfig, if (sendZkNotification)

Re: [PR] KAFKA-15574; [1/N]: Client state machine updates [kafka]

2023-11-07 Thread via GitHub
lianetm commented on code in PR #14690: URL: https://github.com/apache/kafka/pull/14690#discussion_r1385350785 ## clients/src/main/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImpl.java: ## @@ -181,33 +256,465 @@ public void

Re: [PR] KAFKA-15574; [1/N]: Client state machine updates [kafka]

2023-11-07 Thread via GitHub
philipnee commented on code in PR #14690: URL: https://github.com/apache/kafka/pull/14690#discussion_r1383658863 ## clients/src/main/java/org/apache/kafka/clients/consumer/internals/CommitRequestManager.java: ## @@ -166,19 +166,41 @@ private static long findMinTime(final

Re: [PR] KAFKA-14519; [2/N] New coordinator metrics [kafka]

2023-11-07 Thread via GitHub
jeffkbkim commented on code in PR #14387: URL: https://github.com/apache/kafka/pull/14387#discussion_r1385317000 ## group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java: ## @@ -120,6 +121,9 @@ import static

[PR] MINOR: Do not checkpoint standbys when handling corrupted tasks [kafka]

2023-11-07 Thread via GitHub
cadonna opened a new pull request, #14709: URL: https://github.com/apache/kafka/pull/14709 When a task is corrupted, uncorrupted tasks are committed. That is also true for standby tasks. Committing standby tasks actually means that they are checkpointed. When the state updater is

Re: [PR] KAFKA-14519; [2/N] New coordinator metrics [kafka]

2023-11-07 Thread via GitHub
jeffkbkim commented on code in PR #14387: URL: https://github.com/apache/kafka/pull/14387#discussion_r1385307246 ## group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorShard.java: ## @@ -160,11 +161,14 @@ public GroupCoordinatorShard build() {

Re: [PR] KAFKA-15774: refactor windowed stores to use StoreFactory [kafka]

2023-11-07 Thread via GitHub
agavra commented on code in PR #14708: URL: https://github.com/apache/kafka/pull/14708#discussion_r1385298963 ## streams/src/main/java/org/apache/kafka/streams/kstream/internals/TimeWindowStoreMaterializer.java: ## @@ -0,0 +1,133 @@ +/* + * Licensed to the Apache Software

[PR] KAFKA-15774: refactor windowed stores to use StoreFactory [kafka]

2023-11-07 Thread via GitHub
agavra opened a new pull request, #14708: URL: https://github.com/apache/kafka/pull/14708 This is a follow up from #14659 that ports the windowed classes to use the `StoreFactory` abstraction as well. There's a side benefit of not duplicating the materialization code twice for each

[jira] [Updated] (KAFKA-15575) Prevent Connectors from exceeding tasks.max configuration

2023-11-07 Thread Chris Egerton (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15575?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Chris Egerton updated KAFKA-15575: -- Labels: kip (was: ) > Prevent Connectors from exceeding tasks.max configuration >

[jira] [Assigned] (KAFKA-15575) Prevent Connectors from exceeding tasks.max configuration

2023-11-07 Thread Chris Egerton (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15575?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Chris Egerton reassigned KAFKA-15575: - Assignee: Chris Egerton > Prevent Connectors from exceeding tasks.max configuration >

Re: [PR] MINOR: add clarification about log compaction [kafka]

2023-11-07 Thread via GitHub
AyoubOm commented on code in PR #14701: URL: https://github.com/apache/kafka/pull/14701#discussion_r1385285680 ## docs/streams/architecture.html: ## @@ -148,7 +148,8 @@ Log compaction is enabled on the changelog topics so that old data can be purged safely to prevent the

[jira] [Commented] (KAFKA-15020) integration.kafka.server.FetchFromFollowerIntegrationTest.testRackAwareRangeAssignor test is flaky

2023-11-07 Thread Kirk True (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15020?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17783708#comment-17783708 ] Kirk True commented on KAFKA-15020: --- According to [historical test runs over the last 28

Re: [PR] KAFKA-14519; [2/N] New coordinator metrics [kafka]

2023-11-07 Thread via GitHub
jeffkbkim commented on PR #14387: URL: https://github.com/apache/kafka/pull/14387#issuecomment-1799261509 > Also @jeffkbkim can you check the build? Seems like something is off. thanks, i'll take a look > Ah got it, so whether we are adding new or loading from the log we will

[jira] [Comment Edited] (KAFKA-13530) Flaky test ReplicaManagerTest

2023-11-07 Thread Kirk True (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-13530?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17783691#comment-17783691 ] Kirk True edited comment on KAFKA-13530 at 11/7/23 4:42 PM: Out of the last

[jira] [Commented] (KAFKA-13530) Flaky test ReplicaManagerTest

2023-11-07 Thread Kirk True (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-13530?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17783691#comment-17783691 ] Kirk True commented on KAFKA-13530: --- Out of the last 333 runs on trunk, there were no failures or

Re: [PR] KAFKA-15661: KIP-951: Server side changes [kafka]

2023-11-07 Thread via GitHub
jolshan commented on code in PR #1: URL: https://github.com/apache/kafka/pull/1#discussion_r1385219521 ## core/src/main/scala/kafka/server/KafkaApis.scala: ## @@ -562,6 +562,26 @@ class KafkaApis(val requestChannel: RequestChannel, } } + case class

[jira] [Commented] (KAFKA-13530) Flaky test ReplicaManagerTest

2023-11-07 Thread Kirk True (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-13530?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17783687#comment-17783687 ] Kirk True commented on KAFKA-13530: --- I also can't reproduce the flakiness on latest trunk, FWIW. I've

Re: [PR] KAFKA-15583: Enforce HWM advance only if partition is not under-min-ISR [kafka]

2023-11-07 Thread via GitHub
mumrah merged PR #14594: URL: https://github.com/apache/kafka/pull/14594 -- 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-15661: KIP-951: Server side changes [kafka]

2023-11-07 Thread via GitHub
chb2ab commented on code in PR #1: URL: https://github.com/apache/kafka/pull/1#discussion_r1385095201 ## core/src/main/scala/kafka/server/KafkaApis.scala: ## @@ -562,6 +562,26 @@ class KafkaApis(val requestChannel: RequestChannel, } } + case class

Re: [PR] KAFKA-15574; [1/N]: Client state machine updates [kafka]

2023-11-07 Thread via GitHub
lianetm commented on code in PR #14690: URL: https://github.com/apache/kafka/pull/14690#discussion_r1385085418 ## clients/src/main/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImpl.java: ## @@ -181,33 +262,465 @@ public void

Re: [PR] KAFKA-15574; [1/N]: Client state machine updates [kafka]

2023-11-07 Thread via GitHub
lianetm commented on code in PR #14690: URL: https://github.com/apache/kafka/pull/14690#discussion_r1385082158 ## clients/src/main/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImpl.java: ## @@ -181,33 +307,469 @@ public void

Re: [PR] KAFKA-15574; [1/N]: Client state machine updates [kafka]

2023-11-07 Thread via GitHub
lianetm commented on code in PR #14690: URL: https://github.com/apache/kafka/pull/14690#discussion_r1385059304 ## clients/src/main/java/org/apache/kafka/clients/consumer/internals/PrototypeAsyncConsumer.java: ## @@ -843,7 +845,15 @@ private void

Re: [PR] KAFKA-15750: KRaft support in KafkaMetricReporterExceptionHandlingTest [kafka]

2023-11-07 Thread via GitHub
linzihao1999 commented on code in PR #14707: URL: https://github.com/apache/kafka/pull/14707#discussion_r1385018004 ## core/src/test/scala/unit/kafka/server/KafkaMetricReporterExceptionHandlingTest.scala: ## @@ -50,19 +52,30 @@ class KafkaMetricReporterExceptionHandlingTest

Re: [PR] KAFKA-15750: KRaft support in KafkaMetricReporterExceptionHandlingTest [kafka]

2023-11-07 Thread via GitHub
linzihao1999 commented on code in PR #14707: URL: https://github.com/apache/kafka/pull/14707#discussion_r1385016361 ## core/src/test/scala/unit/kafka/server/KafkaMetricReporterExceptionHandlingTest.scala: ## @@ -50,19 +52,30 @@ class KafkaMetricReporterExceptionHandlingTest

Re: [PR] KAFKA-15750: KRaft support in KafkaMetricReporterExceptionHandlingTest [kafka]

2023-11-07 Thread via GitHub
ex172000 commented on code in PR #14707: URL: https://github.com/apache/kafka/pull/14707#discussion_r1385007171 ## core/src/test/scala/unit/kafka/server/KafkaMetricReporterExceptionHandlingTest.scala: ## @@ -50,19 +52,30 @@ class KafkaMetricReporterExceptionHandlingTest extends

Re: [PR] MINOR: add clarification about log compaction [kafka]

2023-11-07 Thread via GitHub
ex172000 commented on code in PR #14701: URL: https://github.com/apache/kafka/pull/14701#discussion_r1384984849 ## docs/streams/architecture.html: ## @@ -148,7 +148,8 @@ Log compaction is enabled on the changelog topics so that old data can be purged safely to prevent the

Re: [PR] MINOR: Enable kraft test in kafka.api and kafka.network [kafka]

2023-11-07 Thread via GitHub
tinaselenge commented on code in PR #14595: URL: https://github.com/apache/kafka/pull/14595#discussion_r1384847563 ## core/src/test/scala/integration/kafka/api/ProducerSendWhileDeletionTest.scala: ## @@ -1,83 +1,87 @@ /** - * Licensed to the Apache Software Foundation (ASF)

Re: [PR] KAFKA-15750: KRaft support in KafkaMetricReporterExceptionHandlingTest [kafka]

2023-11-07 Thread via GitHub
tinaselenge commented on code in PR #14707: URL: https://github.com/apache/kafka/pull/14707#discussion_r1384937097 ## core/src/test/scala/unit/kafka/server/KafkaMetricReporterExceptionHandlingTest.scala: ## @@ -50,19 +52,30 @@ class KafkaMetricReporterExceptionHandlingTest

Re: [PR] KAFKA-15277: Design & implement support for internal Consumer delegates [kafka]

2023-11-07 Thread via GitHub
dajac commented on code in PR #14670: URL: https://github.com/apache/kafka/pull/14670#discussion_r1384566593 ## clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerDelegate.java: ## @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF)

Re: [PR] KAFKA-15750: KRaft support in KafkaMetricReporterExceptionHandlingTest [kafka]

2023-11-07 Thread via GitHub
linzihao1999 commented on code in PR #14707: URL: https://github.com/apache/kafka/pull/14707#discussion_r1384872182 ## core/src/test/scala/unit/kafka/server/KafkaMetricReporterExceptionHandlingTest.scala: ## @@ -50,19 +52,30 @@ class KafkaMetricReporterExceptionHandlingTest

[jira] [Assigned] (KAFKA-15707) KRaft support in TopicBasedRemoteLogMetadataManagerHarness

2023-11-07 Thread Zihao Lin (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15707?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Zihao Lin reassigned KAFKA-15707: - Assignee: Zihao Lin > KRaft support in TopicBasedRemoteLogMetadataManagerHarness >

Re: [PR] MINOR: Enable kraft test in kafka.api and kafka.network [kafka]

2023-11-07 Thread via GitHub
tinaselenge commented on code in PR #14595: URL: https://github.com/apache/kafka/pull/14595#discussion_r1384847563 ## core/src/test/scala/integration/kafka/api/ProducerSendWhileDeletionTest.scala: ## @@ -1,83 +1,87 @@ /** - * Licensed to the Apache Software Foundation (ASF)

Re: [PR] KAFKA-15750: KRaft support in KafkaMetricReporterExceptionHandlingTest [kafka]

2023-11-07 Thread via GitHub
tinaselenge commented on code in PR #14707: URL: https://github.com/apache/kafka/pull/14707#discussion_r1384842111 ## core/src/test/scala/unit/kafka/server/KafkaMetricReporterExceptionHandlingTest.scala: ## @@ -50,19 +52,30 @@ class KafkaMetricReporterExceptionHandlingTest

  1   2   >