Re: [PR] KAFKA-14485: Move LogCleaner to storage module [kafka]

2025-04-07 Thread via GitHub
wernerdv commented on code in PR #19387: URL: https://github.com/apache/kafka/pull/19387#discussion_r2032487977 ## build.gradle: ## @@ -3737,6 +3737,7 @@ project(':connect:mirror') { testImplementation project(':core') testImplementation project(':test-common:test-comm

Re: [PR] KAFKA-19101 Remove ControllerMutationQuotaManager#throttleTimeMs unused parameter [kafka]

2025-04-07 Thread via GitHub
FrankYang0529 commented on code in PR #19410: URL: https://github.com/apache/kafka/pull/19410#discussion_r2032444160 ## core/src/main/scala/kafka/server/ControllerMutationQuotaManager.scala: ## @@ -63,7 +63,7 @@ abstract class AbstractControllerMutationQuota(private val time: T

Re: [PR] KAFKA-7061: KIP-280 Enhanced log compaction [kafka]

2025-04-07 Thread via GitHub
senthilm-ms commented on PR #8103: URL: https://github.com/apache/kafka/pull/8103#issuecomment-2785163121 Got it and I see the interests from many in the community... I will spend some time by end of April & early May and send the update PR. Thanks, Senthil From: Matth

Re: [PR] MINOR: Modify KafkaEventQueue VoidEvent to as singleton and use more proper function interface [kafka]

2025-04-07 Thread via GitHub
gongxuanzhang commented on PR #19356: URL: https://github.com/apache/kafka/pull/19356#issuecomment-2785145813 @chia7712 WDYT -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment

Re: [PR] KAFKA-19100: Use ProcessRole.toString instead of String in AclApis [kafka]

2025-04-07 Thread via GitHub
gongxuanzhang commented on code in PR #19406: URL: https://github.com/apache/kafka/pull/19406#discussion_r2032322050 ## core/src/main/scala/kafka/server/ControllerApis.scala: ## @@ -84,7 +85,7 @@ class ControllerApis( val configHelper = new ConfigHelper(metadataCache, config,

Re: [PR] Change Controller Mutation Quota Implementation [kafka]

2025-04-07 Thread via GitHub
github-actions[bot] commented on PR #19318: URL: https://github.com/apache/kafka/pull/19318#issuecomment-2785121312 A label of 'needs-attention' was automatically added to this PR in order to raise the attention of the committers. Once this issue has been triaged, the `triage` label s

Re: [PR] KAFKA-18874: Fix KRaft controller registration retry on timeout [kafka]

2025-04-07 Thread via GitHub
github-actions[bot] commented on PR #19321: URL: https://github.com/apache/kafka/pull/19321#issuecomment-2785121287 A label of 'needs-attention' was automatically added to this PR in order to raise the attention of the committers. Once this issue has been triaged, the `triage` label s

Re: [PR] KAFKA-16729: Support isolation level for share consumer [kafka]

2025-04-07 Thread via GitHub
adixitconfluent commented on code in PR #19261: URL: https://github.com/apache/kafka/pull/19261#discussion_r2031767657 ## clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/consumer/ShareConsumerTest.java: ## @@ -2090,6 +2091,307 @@ public void testComplexS

Re: [PR] KAFKA-7061: KIP-280 Enhanced log compaction [kafka]

2025-04-07 Thread via GitHub
mjsax commented on PR #8103: URL: https://github.com/apache/kafka/pull/8103#issuecomment-2785005543 Just a question of review capacity I assume -- I think it would still be valuable to complete this, but somebody must have time to do the work, plus some committer agree to help reviewing...

Re: [PR] MINOR: remove transform and through from repartition description [kafka]

2025-04-07 Thread via GitHub
mjsax commented on code in PR #19291: URL: https://github.com/apache/kafka/pull/19291#discussion_r2032232490 ## docs/streams/developer-guide/dsl-api.html: ## @@ -764,10 +764,10 @@ Manually trigger repartitioning of the stream with desired number of partitions. (details) -

[jira] [Assigned] (KAFKA-19102) Enhance the docs of group.coordinator.append.linger.ms

2025-04-07 Thread Jira
[ https://issues.apache.org/jira/browse/KAFKA-19102?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] 黃竣陽 reassigned KAFKA-19102: --- Assignee: 黃竣陽 (was: Chia-Ping Tsai) > Enhance the docs of group.coordinator.append.linger.ms > ---

Re: [PR] MINOR: use enum map for error counts map [kafka]

2025-04-07 Thread via GitHub
mjsax commented on PR #19314: URL: https://github.com/apache/kafka/pull/19314#issuecomment-2784956048 This one is still small enough. -- No reason to split it. -- There is no fixed rule. It always depends always on the ticket, but I usually try to keep it below 500 LOC -- if larger, there s

[jira] [Updated] (KAFKA-16996) The leastLoadedNode() function in kafka-client may choose a faulty node during the consumer thread starting and meanwhile one of the KAFKA server node is dead.

2025-04-07 Thread Lianet Magrans (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16996?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Lianet Magrans updated KAFKA-16996: --- Component/s: consumer > The leastLoadedNode() function in kafka-client may choose a faulty n

Re: [PR] KAFKA-16729: Support isolation level for share consumer [kafka]

2025-04-07 Thread via GitHub
junrao commented on code in PR #19261: URL: https://github.com/apache/kafka/pull/19261#discussion_r2031937222 ## clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/consumer/ShareConsumerTest.java: ## @@ -2090,6 +2091,307 @@ public void testComplexShareConsu

[PR] MINOR: Minor tidying in GroupMetadataManager [kafka]

2025-04-07 Thread via GitHub
AndrewJSchofield opened a new pull request, #19411: URL: https://github.com/apache/kafka/pull/19411 Some trivial tidying up in GroupMetadataManager. -- 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 t

[jira] [Updated] (KAFKA-18962) StateRestoreListener onBatchRestored method is called with the totalRestored on GlobalStateStore reprocess

2025-04-07 Thread Matthias J. Sax (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-18962?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax updated KAFKA-18962: Affects Version/s: 3.8.0 (was: 3.9.0) > StateRestoreListener on

Re: [PR] KAFKA-18962: Fix onBatchRestored call in GlobalStateManagerImpl [kafka]

2025-04-07 Thread via GitHub
mjsax commented on code in PR #19188: URL: https://github.com/apache/kafka/pull/19188#discussion_r2032155035 ## clients/src/main/java/org/apache/kafka/clients/consumer/MockConsumer.java: ## @@ -294,13 +306,17 @@ public synchronized ConsumerRecords poll(final Duration timeout) {

[jira] [Updated] (KAFKA-16394) ForeignKey LEFT join propagates null value on foreignKey change

2025-04-07 Thread Matthias J. Sax (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16394?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax updated KAFKA-16394: Fix Version/s: (was: 3.9.1) > ForeignKey LEFT join propagates null value on foreignKey

Re: [PR] KAFKA-19100: Use ProcessRole.toString instead of String in AclApis [kafka]

2025-04-07 Thread via GitHub
mimaison commented on code in PR #19406: URL: https://github.com/apache/kafka/pull/19406#discussion_r2031923532 ## core/src/main/scala/kafka/server/ControllerApis.scala: ## @@ -84,7 +85,7 @@ class ControllerApis( val configHelper = new ConfigHelper(metadataCache, config, meta

Re: [PR] KAFKA-19047: Broker registrations are slow if previously fenced or shutdown [kafka]

2025-04-07 Thread via GitHub
jsancio commented on code in PR #19296: URL: https://github.com/apache/kafka/pull/19296#discussion_r2031970291 ## metadata/src/main/java/org/apache/kafka/controller/ClusterControlManager.java: ## @@ -353,7 +353,7 @@ public ControllerResult registerBroker( if (existing

[jira] [Created] (KAFKA-19104) Support metadata version downgrade in Kafka

2025-04-07 Thread Colin McCabe (Jira)
Colin McCabe created KAFKA-19104: Summary: Support metadata version downgrade in Kafka Key: KAFKA-19104 URL: https://issues.apache.org/jira/browse/KAFKA-19104 Project: Kafka Issue Type: Bug

Re: [PR] KAFKA-19047: Broker registrations are slow if previously fenced or shutdown [kafka]

2025-04-07 Thread via GitHub
ahuang98 commented on code in PR #19296: URL: https://github.com/apache/kafka/pull/19296#discussion_r2031718807 ## metadata/src/main/java/org/apache/kafka/controller/ClusterControlManager.java: ## @@ -353,7 +353,7 @@ public ControllerResult registerBroker( if (existing

Re: [PR] KAFKA-19090: Move DelayedFuture and DelayedFuturePurgatory to server-common module [kafka]

2025-04-07 Thread via GitHub
mimaison commented on PR #19390: URL: https://github.com/apache/kafka/pull/19390#issuecomment-2784473414 Should these classes go to the `server` module instead? They only seem to be used by `AclApis`. -- This is an automated message from the Apache Git Service. To respond to the message,

Re: [PR] KAFKA-16729: Support isolation level for share consumer [kafka]

2025-04-07 Thread via GitHub
adixitconfluent commented on code in PR #19261: URL: https://github.com/apache/kafka/pull/19261#discussion_r2031778027 ## clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/consumer/ShareConsumerTest.java: ## @@ -2090,6 +2091,307 @@ public void testComplexS

Re: [PR] KAFKA-16729: Support isolation level for share consumer [kafka]

2025-04-07 Thread via GitHub
adixitconfluent commented on code in PR #19261: URL: https://github.com/apache/kafka/pull/19261#discussion_r2031745262 ## core/src/test/java/kafka/server/share/SharePartitionTest.java: ## @@ -6671,15 +6693,441 @@ private String assertionFailedMessage(SharePartition sharePartiti

Re: [PR] KAFKA-16729: Support isolation level for share consumer [kafka]

2025-04-07 Thread via GitHub
adixitconfluent commented on code in PR #19261: URL: https://github.com/apache/kafka/pull/19261#discussion_r2031757437 ## core/src/test/java/kafka/server/share/SharePartitionTest.java: ## @@ -6671,15 +6693,441 @@ private String assertionFailedMessage(SharePartition sharePartiti

[jira] [Commented] (KAFKA-19103) Remove OffsetConfig

2025-04-07 Thread Bolin Lin (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-19103?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17941677#comment-17941677 ] Bolin Lin commented on KAFKA-19103: --- Hi [~chia7712],  ** I just jumped in—could you pl

[jira] [Created] (KAFKA-19103) Remove OffsetConfig

2025-04-07 Thread Chia-Ping Tsai (Jira)
Chia-Ping Tsai created KAFKA-19103: -- Summary: Remove OffsetConfig Key: KAFKA-19103 URL: https://issues.apache.org/jira/browse/KAFKA-19103 Project: Kafka Issue Type: Improvement R

Re: [PR] KAFKA-16729: Support isolation level for share consumer [kafka]

2025-04-07 Thread via GitHub
junrao commented on code in PR #19261: URL: https://github.com/apache/kafka/pull/19261#discussion_r2031538464 ## core/src/main/java/kafka/server/share/SharePartition.java: ## @@ -2488,6 +2509,190 @@ private long startOffsetDuringInitialization(long partitionDataStartOffset) thr

Re: [PR] MINOR: use enum map for error counts map [kafka]

2025-04-07 Thread via GitHub
lorcanj commented on PR #19314: URL: https://github.com/apache/kafka/pull/19314#issuecomment-2784070246 Hi @mjsax, I noticed you commented on a ticket that I raised which mentioned not combining too many changes into 1 PR. Could you advise whether it would be advisable to split this up into

[jira] [Created] (KAFKA-19101) Remove ControllerMutationQuotaManager#throttleTimeMs unused parameter

2025-04-07 Thread Jira
黃竣陽 created KAFKA-19101: --- Summary: Remove ControllerMutationQuotaManager#throttleTimeMs unused parameter Key: KAFKA-19101 URL: https://issues.apache.org/jira/browse/KAFKA-19101 Project: Kafka Issue Ty

Re: [PR] KAFKA-19054: StreamThread exception handling with SHUTDOWN_APPLICATION may trigger a tight loop with MANY logs [kafka]

2025-04-07 Thread via GitHub
apalan60 commented on PR #19394: URL: https://github.com/apache/kafka/pull/19394#issuecomment-2783979984 @FrankYang0529 Thanks for your review. I've added a test. PTAL, Thanks! -- This is an automated message from the Apache Git Service. To respond to the message, please log on

Re: [PR] KAFKA-19030: Remove metricNamePrefix from RequestChannel [kafka]

2025-04-07 Thread via GitHub
Parkerhiphop commented on code in PR #19374: URL: https://github.com/apache/kafka/pull/19374#discussion_r2030194156 ## core/src/main/scala/kafka/network/SocketServer.scala: ## @@ -97,7 +97,7 @@ class SocketServer( private val memoryPool = if (config.queuedMaxBytes > 0) new S

[PR] KAFKA-19101 Remove ControllerMutationQuotaManager#throttleTimeMs unused parameter [kafka]

2025-04-07 Thread via GitHub
m1a2st opened a new pull request, #19410: URL: https://github.com/apache/kafka/pull/19410 It seems `timeMs` this parameter never used in Kafka project, the method init commit is https://github.com/apache/kafka/commit/b5f90daf13b4945305951ca0eecdb454a4dcafc2 -- This is an automated messag

[jira] [Updated] (KAFKA-19097) Fix order of arguments to assertEquals

2025-04-07 Thread Matthias J. Sax (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-19097?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax updated KAFKA-19097: Component/s: streams unit tests > Fix order of arguments to assertEquals

Re: [PR] KAFKA-19030: Remove metricNamePrefix from RequestChannel [kafka]

2025-04-07 Thread via GitHub
Parkerhiphop commented on code in PR #19374: URL: https://github.com/apache/kafka/pull/19374#discussion_r2031577526 ## core/src/main/scala/kafka/network/SocketServer.scala: ## @@ -97,7 +97,7 @@ class SocketServer( private val memoryPool = if (config.queuedMaxBytes > 0) new S

Re: [PR] KAFKA-19030: Remove metricNamePrefix from RequestChannel [kafka]

2025-04-07 Thread via GitHub
Parkerhiphop commented on code in PR #19374: URL: https://github.com/apache/kafka/pull/19374#discussion_r2030194156 ## core/src/main/scala/kafka/network/SocketServer.scala: ## @@ -97,7 +97,7 @@ class SocketServer( private val memoryPool = if (config.queuedMaxBytes > 0) new S

[jira] [Resolved] (KAFKA-19098) Remove lastOffset from PartitionResponse

2025-04-07 Thread Chia-Ping Tsai (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-19098?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Chia-Ping Tsai resolved KAFKA-19098. Fix Version/s: 4.1.0 Resolution: Fixed > Remove lastOffset from PartitionResponse >

Re: [PR] KAFKA-16394: Fix null propagation in foreign key join result [kafka]

2025-04-07 Thread via GitHub
mjsax commented on PR #15607: URL: https://github.com/apache/kafka/pull/15607#issuecomment-2783873590 As discussed via DM on some other channel, I think the original issue of duplicate tombstones was already fixed via https://github.com/apache/kafka/pull/19005 I think we should re-pu

Re: [PR] KAFKA-19030: Remove metricNamePrefix from RequestChannel [kafka]

2025-04-07 Thread via GitHub
Parkerhiphop commented on code in PR #19374: URL: https://github.com/apache/kafka/pull/19374#discussion_r2031557837 ## core/src/main/scala/kafka/network/SocketServer.scala: ## @@ -495,8 +491,7 @@ private[kafka] abstract class Acceptor(val socketServer: SocketServer, val sh

Re: [PR] MINOR: Cleanup Core Module [kafka]

2025-04-07 Thread via GitHub
AndrewJSchofield merged PR #19372: URL: https://github.com/apache/kafka/pull/19372 -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: jira-unsubscr...@kafk

Re: [PR] KAFKA-16718-2/n: KafkaAdminClient and GroupCoordinator implementation for DeleteShareGroupOffsets RPC [kafka]

2025-04-07 Thread via GitHub
AndrewJSchofield commented on code in PR #18976: URL: https://github.com/apache/kafka/pull/18976#discussion_r2031381756 ## group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorService.java: ## @@ -1508,6 +1555,110 @@ private CompletableFuture dele

Re: [PR] KAFKA-15853: Move ShareCoordinatorConfig and GroupCoordinatorConfig out of KafkaConfig [kafka]

2025-04-07 Thread via GitHub
dajac commented on code in PR #19409: URL: https://github.com/apache/kafka/pull/19409#discussion_r2031502146 ## core/src/main/scala/kafka/server/KafkaConfig.scala: ## @@ -194,16 +193,9 @@ class KafkaConfig private(doLog: Boolean, val props: util.Map[_, _]) private val _quoru

Re: [PR] KAFKA-19060 Documented null edge cases in the Clients API JavaDoc [kafka]

2025-04-07 Thread via GitHub
m1a2st commented on PR #19393: URL: https://github.com/apache/kafka/pull/19393#issuecomment-2783697038 Hello @kirktrue, if you have free cycle, PTAL -- 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 t

[PR] KAFKA-15853: Move ShareCoordinatorConfig and GroupCoordinatorConfig out of KafkaConfig [kafka]

2025-04-07 Thread via GitHub
FrankYang0529 opened a new pull request, #19409: URL: https://github.com/apache/kafka/pull/19409 Since GroupCoordinatorConfig and GroupCoordinatorConfig are not reconfigurable, they don't need to be integrated in KafkaConfig. -- This is an automated message from the Apache Git Service. To

Re: [PR] KAFKA-18894: Add KIP-877 support for ConfigProvider [kafka]

2025-04-07 Thread via GitHub
m1a2st commented on PR #19397: URL: https://github.com/apache/kafka/pull/19397#issuecomment-2783653368 Thanks for this patch, please use `./gradlew spotlessApply` fix build error -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub

Re: [PR] MINOR: Only send endpoints to partitions on changes [kafka]

2025-04-07 Thread via GitHub
bbejeck commented on code in PR #19407: URL: https://github.com/apache/kafka/pull/19407#discussion_r2031311815 ## group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java: ## @@ -513,6 +513,8 @@ GroupMetadataManager build() { */ pri

Re: [PR] KAFKA-19060 Documented null edge cases in the Clients API JavaDoc [kafka]

2025-04-07 Thread via GitHub
m1a2st commented on code in PR #19393: URL: https://github.com/apache/kafka/pull/19393#discussion_r2031149018 ## clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java: ## @@ -1658,7 +1660,8 @@ public Map beginningOffsets(Collection par * @see #seekToE

Re: [PR] KAFKA-18888: Add KIP-877 support to Authorizer [kafka]

2025-04-07 Thread via GitHub
m1a2st commented on code in PR #19050: URL: https://github.com/apache/kafka/pull/19050#discussion_r2031390658 ## group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java: ## @@ -3284,7 +3285,7 @@ private boolean maybeUpdateRegularExpressions(

Re: [PR] MINOR: Only send endpoints to partitions on changes [kafka]

2025-04-07 Thread via GitHub
bbejeck commented on PR #19407: URL: https://github.com/apache/kafka/pull/19407#issuecomment-2783476733 @aliehsaeedii for review -- 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 comme

Re: [PR] KAFKA-18888: Add KIP-877 support to Authorizer [kafka]

2025-04-07 Thread via GitHub
m1a2st commented on code in PR #19050: URL: https://github.com/apache/kafka/pull/19050#discussion_r2031317020 ## group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java: ## @@ -3284,7 +3285,7 @@ private boolean maybeUpdateRegularExpressions(

[PR] MINOR: move some class to server from server_common [kafka]

2025-04-07 Thread via GitHub
gongxuanzhang opened a new pull request, #19408: URL: https://github.com/apache/kafka/pull/19408 about https://github.com/apache/kafka/pull/19226#issuecomment-2783370917 -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and

Re: [PR] KAFKA-18913: Removing _state.updater.enabled_ flag through the Stream… [kafka]

2025-04-07 Thread via GitHub
cadonna commented on code in PR #19275: URL: https://github.com/apache/kafka/pull/19275#discussion_r2031317747 ## streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/SmokeTestDriverIntegrationTest.java: ## @@ -99,8 +99,8 @@ SmokeTestDriver.VerificationRe

[jira] [Assigned] (KAFKA-17777) Flaky KafkaConsumerTest testReturnRecordsDuringRebalance

2025-04-07 Thread Kirk True (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-1?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Kirk True reassigned KAFKA-1: - Assignee: (was: Kirk True) > Flaky KafkaConsumerTest testReturnRecordsDuringRebalance > ---

[PR] MINOR: Only send endpoints to partitions on changes [kafka]

2025-04-07 Thread via GitHub
bbejeck opened a new pull request, #19407: URL: https://github.com/apache/kafka/pull/19407 This PR updates the StreamGroup heart beat response to only include the `endpointToPartitions` information after there has been a task assignment change and all members have successfully migrated to t

Re: [PR] KAFKA-19004:Move DelayedDeleteRecords to server-common module [kafka]

2025-04-07 Thread via GitHub
gongxuanzhang commented on PR #19226: URL: https://github.com/apache/kafka/pull/19226#issuecomment-2783399502 @mimaison @chia7712 Thanks for your comments I will create a MINOR PR to address it -- This is an automated message from the Apache Git Service. To respond to the message

[PR] KAFKA-19100: Use ProcessRole.toString instead of String in AclApis [kafka]

2025-04-07 Thread via GitHub
gongxuanzhang opened a new pull request, #19406: URL: https://github.com/apache/kafka/pull/19406 like title about jira https://issues.apache.org/jira/browse/KAFKA-19100 -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and us

Re: [PR] KAFKA-19004:Move DelayedDeleteRecords to server-common module [kafka]

2025-04-07 Thread via GitHub
chia7712 commented on PR #19226: URL: https://github.com/apache/kafka/pull/19226#issuecomment-2783375231 @gongxuanzhang WDYT? If you agree @mimaison's comment, could you please create a MINOR to address it? -- This is an automated message from the Apache Git Service. To respond to the me

Re: [PR] KAFKA-19004:Move DelayedDeleteRecords to server-common module [kafka]

2025-04-07 Thread via GitHub
mimaison commented on PR #19226: URL: https://github.com/apache/kafka/pull/19226#issuecomment-2783370917 Yes -- 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] [Assigned] (KAFKA-19100) Use ProcessRole.toString instead of String in AclApis

2025-04-07 Thread xuanzhang gong (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-19100?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] xuanzhang gong reassigned KAFKA-19100: -- Assignee: xuanzhang gong > Use ProcessRole.toString instead of String in AclApis > --

[jira] [Created] (KAFKA-19100) Use ProcessRole.toString instead of String in AclApis

2025-04-07 Thread Jira
黃竣陽 created KAFKA-19100: --- Summary: Use ProcessRole.toString instead of String in AclApis Key: KAFKA-19100 URL: https://issues.apache.org/jira/browse/KAFKA-19100 Project: Kafka Issue Type: Task

Re: [PR] KAFKA-19077: Propagate shutdownRequested field [kafka]

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

Re: [PR] KAFKA-18796-3: Increased the default acquisition lock timeout in SharePartition from 100ms to 2 seconds [kafka]

2025-04-07 Thread via GitHub
chirag-wadhwa5 commented on PR #19405: URL: https://github.com/apache/kafka/pull/19405#issuecomment-2783292100 Thanks for the review. Answering the questions - 1. The flakiness could not be reproduced locally in the first place, so there is no proof of that. Honestly reproducing the error

Re: [PR] KAFKA-19004:Move DelayedDeleteRecords to server-common module [kafka]

2025-04-07 Thread via GitHub
chia7712 commented on PR #19226: URL: https://github.com/apache/kafka/pull/19226#issuecomment-2783237052 > RemoteLogManager only uses DelayedRemoteListOffsets hence why we put that in the storage module. But as far as I can tell other purgatory code could be in server. `RemoteLogMana

[PR] KAFKA-18796-3: Increased the default acquisition lock timeout in SharePartition from 100ms to 2 seconds [kafka]

2025-04-07 Thread via GitHub
chirag-wadhwa5 opened a new pull request, #19405: URL: https://github.com/apache/kafka/pull/19405 This PR increases the default acquisition lock timeout in SharePartitionTest from 100ms to 2 seconds in an attempt to remove flakiness from the respective tests. The test reports suggest some o

Re: [PR] KAFKA-19004:Move DelayedDeleteRecords to server-common module [kafka]

2025-04-07 Thread via GitHub
mimaison commented on PR #19226: URL: https://github.com/apache/kafka/pull/19226#issuecomment-2783221120 `RemoteLogManager` only uses `DelayedRemoteListOffsets` hence why we put that in the `storage` module. But as far as I can tell other purgatory code could be in `server`. -- This is a

Re: [PR] KAFKA-19030: Remove metricNamePrefix from RequestChannel [kafka]

2025-04-07 Thread via GitHub
m1a2st commented on code in PR #19374: URL: https://github.com/apache/kafka/pull/19374#discussion_r2031136549 ## core/src/main/scala/kafka/network/SocketServer.scala: ## @@ -495,8 +491,7 @@ private[kafka] abstract class Acceptor(val socketServer: SocketServer, val shouldRu

Re: [PR] KAFKA-19027: Replace ConsumerGroupCommandTestUtils#generator by ClusterTestDefaults [kafka]

2025-04-07 Thread via GitHub
m1a2st commented on code in PR #19347: URL: https://github.com/apache/kafka/pull/19347#discussion_r2031081161 ## tools/src/test/java/org/apache/kafka/tools/consumer/group/ConsumerGroupCommandTestUtils.java: ## @@ -38,12 +34,6 @@ import java.util.function.Consumer; import java.

Re: [PR] KAFKA-19027: Replace ConsumerGroupCommandTestUtils#generator by ClusterTestDefaults [kafka]

2025-04-07 Thread via GitHub
chia7712 commented on code in PR #19347: URL: https://github.com/apache/kafka/pull/19347#discussion_r2031094554 ## tools/src/test/java/org/apache/kafka/tools/consumer/group/ConsumerGroupCommandTestUtils.java: ## @@ -38,12 +34,6 @@ import java.util.function.Consumer; import jav

Re: [PR] KAFKA-19027: Replace ConsumerGroupCommandTestUtils#generator by ClusterTestDefaults [kafka]

2025-04-07 Thread via GitHub
Rancho-7 commented on code in PR #19347: URL: https://github.com/apache/kafka/pull/19347#discussion_r2031070243 ## tools/src/test/java/org/apache/kafka/tools/consumer/group/DescribeConsumerGroupTest.java: ## @@ -66,13 +68,28 @@ import java.util.stream.IntStream; import java.ut

Re: [PR] KAFKA-19027: Replace ConsumerGroupCommandTestUtils#generator by ClusterTestDefaults [kafka]

2025-04-07 Thread via GitHub
chia7712 commented on code in PR #19347: URL: https://github.com/apache/kafka/pull/19347#discussion_r2031064108 ## tools/src/test/java/org/apache/kafka/tools/consumer/group/DescribeConsumerGroupTest.java: ## @@ -66,13 +68,28 @@ import java.util.stream.IntStream; import java.ut

[jira] [Commented] (KAFKA-19099) Remove GroupSyncKey, GroupJoinKey, and MemberKey

2025-04-07 Thread Nick Guo (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-19099?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17941543#comment-17941543 ] Nick Guo commented on KAFKA-19099: -- Hi [~chia7712] ,I would like to take this issue.Tha

[jira] [Assigned] (KAFKA-19099) Remove GroupSyncKey, GroupJoinKey, and MemberKey

2025-04-07 Thread Nick Guo (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-19099?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Nick Guo reassigned KAFKA-19099: Assignee: Nick Guo (was: Chia-Ping Tsai) > Remove GroupSyncKey, GroupJoinKey, and MemberKey > --

[jira] [Commented] (KAFKA-6629) SegmentedCacheFunctionTest does not cover session window serdes

2025-04-07 Thread Lorcan (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-6629?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17941541#comment-17941541 ] Lorcan commented on KAFKA-6629: --- Hi [~guozhang], I've created a PR for this change: https:

[jira] [Created] (KAFKA-19099) Remove GroupSyncKey, GroupJoinKey, and MemberKey

2025-04-07 Thread Chia-Ping Tsai (Jira)
Chia-Ping Tsai created KAFKA-19099: -- Summary: Remove GroupSyncKey, GroupJoinKey, and MemberKey Key: KAFKA-19099 URL: https://issues.apache.org/jira/browse/KAFKA-19099 Project: Kafka Issue Ty

Re: [PR] KAFKA-19004:Move DelayedDeleteRecords to server-common module [kafka]

2025-04-07 Thread via GitHub
chia7712 commented on PR #19226: URL: https://github.com/apache/kafka/pull/19226#issuecomment-2783031557 > I was looking at https://github.com/apache/kafka/pull/19390 and wondering whether we should put these classes in server instead of server-common as they don't seem to be used elsewhere

Re: [PR] KAFKA-18981: Fix flaky QuorumControllerTest.testMinIsrUpdateWithElr [kafka]

2025-04-07 Thread via GitHub
FrankYang0529 commented on PR #19262: URL: https://github.com/apache/kafka/pull/19262#issuecomment-2783020471 Hi @mumrah, do we have chance to merge this PR? In last 28 days, the QuorumControllerTest#testMinIsrUpdateWithElr has 5% to get flaky result. ![Screenshot 2025-04-07 at 7 33 4

[PR] KAFKA-6629: parameterise SegmentedCacheFunctionTest for session key schemas [kafka]

2025-04-07 Thread via GitHub
lorcanj opened a new pull request, #19404: URL: https://github.com/apache/kafka/pull/19404 Addresses: [KAFKA-6629](https://issues.apache.org/jira/browse/KAFKA-6629) Adds configuration for the SessionKeySchema and parameterises the existing tests so that both WindowKeys and SessionK

Re: [PR] KAFKA-18845: Remove flaky tag on QuorumControllerTest#testUncleanShutdownBrokerElrEnabled [kafka]

2025-04-07 Thread via GitHub
FrankYang0529 commented on PR #19403: URL: https://github.com/apache/kafka/pull/19403#issuecomment-2783002149 ![Screenshot 2025-04-07 at 7 28 55  PM](https://github.com/user-attachments/assets/e2c17ecf-f6da-4f5d-bc3c-b001cf0c2550) -- This is an automated message from the Apache Git Ser

[PR] KAFKA-18845: Remove flaky tag on QuorumControllerTest#testUncleanShutdownBrokerElrEnabled [kafka]

2025-04-07 Thread via GitHub
FrankYang0529 opened a new pull request, #19403: URL: https://github.com/apache/kafka/pull/19403 It has been around two weeks since fixing QuorumControllerTest#testUncleanShutdownBrokerElrEnabled PR https://github.com/apache/kafka/pull/19240 was merged. There is no flaky result after 2025/

Re: [PR] KAFKA-18888: Add KIP-877 support to Authorizer [kafka]

2025-04-07 Thread via GitHub
mimaison commented on code in PR #19050: URL: https://github.com/apache/kafka/pull/19050#discussion_r2031012460 ## metadata/src/main/java/org/apache/kafka/metadata/publisher/AclPublisher.java: ## @@ -103,6 +105,9 @@ public void onMetadataUpdate(MetadataDelta delta, MetadataImag

Re: [PR] KAFKA-19037: Integrate consumer-side code with Streams [kafka]

2025-04-07 Thread via GitHub
cadonna commented on code in PR #19377: URL: https://github.com/apache/kafka/pull/19377#discussion_r2030990356 ## streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/SmokeTestDriverIntegrationTest.java: ## @@ -99,8 +115,17 @@ SmokeTestDriver.Verification

[jira] [Assigned] (KAFKA-18066) Misleading/mismatched StreamThread id in logging

2025-04-07 Thread Uladzislau Blok (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-18066?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Uladzislau Blok reassigned KAFKA-18066: --- Assignee: Uladzislau Blok (was: Peter Lee) > Misleading/mismatched StreamThread id

[jira] [Commented] (KAFKA-18066) Misleading/mismatched StreamThread id in logging

2025-04-07 Thread Uladzislau Blok (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-18066?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17941531#comment-17941531 ] Uladzislau Blok commented on KAFKA-18066: - Hey [~peterxcli], Are you still looki

[jira] [Commented] (KAFKA-18066) Misleading/mismatched StreamThread id in logging

2025-04-07 Thread Peter Lee (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-18066?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17941532#comment-17941532 ] Peter Lee commented on KAFKA-18066: --- Sure > Misleading/mismatched StreamThread id in

Re: [PR] KAFKA-18888: Add KIP-877 support to Authorizer [kafka]

2025-04-07 Thread via GitHub
mimaison commented on code in PR #19050: URL: https://github.com/apache/kafka/pull/19050#discussion_r2030840916 ## group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTestContext.java: ## @@ -505,8 +506,8 @@ public Builder withShareGroupAssign

Re: [PR] KAFKA-19077: Propagate shutdownRequested field [kafka]

2025-04-07 Thread via GitHub
lucasbru commented on code in PR #19359: URL: https://github.com/apache/kafka/pull/19359#discussion_r2030885666 ## group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/StreamsGroupTest.java: ## @@ -1106,4 +1106,33 @@ public void testIsSubscribedToTopic() {

Re: [PR] KAFKA-17639 Add Java 23 to CI [kafka]

2025-04-07 Thread via GitHub
showuon commented on PR #19396: URL: https://github.com/apache/kafka/pull/19396#issuecomment-2782818007 The Jenkins build adds JDK 23 now, and the build completes with 2 flaky tests: https://ci-builds.apache.org/job/Kafka/job/kafka-pr/job/PR-19396/2/ @gharris1727 , please help review.

Re: [PR] KAFKA-19077: Propagate shutdownRequested field [kafka]

2025-04-07 Thread via GitHub
lucasbru commented on code in PR #19359: URL: https://github.com/apache/kafka/pull/19359#discussion_r2030885033 ## group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java: ## @@ -2226,6 +2224,16 @@ private CoordinatorResult stream

Re: [PR] KAFKA-19001: Use streams group-level configurations in heartbeat [kafka]

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

Re: [PR] KAFKA-18888: Add KIP-877 support to Authorizer [kafka]

2025-04-07 Thread via GitHub
mimaison commented on code in PR #19050: URL: https://github.com/apache/kafka/pull/19050#discussion_r2030844573 ## server/src/test/java/org/apache/kafka/server/MonitorablePluginsIntegrationTest.java: ## @@ -0,0 +1,116 @@ +/* + * Licensed to the Apache Software Foundation (ASF) u

Re: [PR] KAFKA-18888: Add KIP-877 support to Authorizer [kafka]

2025-04-07 Thread via GitHub
mimaison commented on code in PR #19050: URL: https://github.com/apache/kafka/pull/19050#discussion_r2030836809 ## group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorShard.java: ## @@ -250,7 +251,7 @@ public GroupCoordinatorShard build() {

[PR] MINOR: Port changes from KAFKA-18569 for ShareConsumers [kafka]

2025-04-07 Thread via GitHub
ShivsundarR opened a new pull request, #19402: URL: https://github.com/apache/kafka/pull/19402 *What* - `ShareConsumers` may wait on an unneeded `FindCoordinator` during `close()`(i.e after the acknowledgements are sent). - This change https://github.com/apache/kafka/pull/18590

Re: [PR] KAFKA-19084: Port KAFKA-16224, KAFKA-16764 for ShareConsumers [kafka]

2025-04-07 Thread via GitHub
AndrewJSchofield merged PR #19369: URL: https://github.com/apache/kafka/pull/19369 -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: jira-unsubscr...@kafk

Re: [PR] MINOR: Add documentation about KIP-405 remote reads serving just one partition per FetchRequest [kafka]

2025-04-07 Thread via GitHub
showuon commented on code in PR #19336: URL: https://github.com/apache/kafka/pull/19336#discussion_r2030785098 ## docs/ops.html: ## @@ -4026,13 +4026,14 @@ Broker Configurations -By default, Kafka server will not enable tiered storage feature. remote.log.storage.system.enabl

[PR] MINOR: Rename RemoteLogStorageManager variable to RemoteStorageManager [kafka]

2025-04-07 Thread via GitHub
stanislavkozlovski opened a new pull request, #19401: URL: https://github.com/apache/kafka/pull/19401 This patch renames the KIP-405 Plugin variable from `remoteLogStorageManager` to `remoteStorageManager`. After [writing about it](https://aiven.io/blog/apache-kafka-tiered-storage-in-depth-

Re: [PR] KAFKA-18402: Optimise of comparison on org.apache.kafka.common.Uuid [kafka]

2025-04-07 Thread via GitHub
bmscomp commented on PR #13627: URL: https://github.com/apache/kafka/pull/13627#issuecomment-2782438397 @divijvaidya @ijuma Still think this is a better implementation, I think what do you think about it ? Or should I close this -- This is an automated message from the Apache Git Servi

Re: [PR] KAFKA-18336: Improve jmh tests on ACL in AuthorizerBenchmark and StandardAuthorizerUpdateBenchmark [kafka]

2025-04-07 Thread via GitHub
ekuvardin commented on PR #18293: URL: https://github.com/apache/kafka/pull/18293#issuecomment-2782433185 @chia7712 Could you please help and review code? Next stage after this MR I would like to rewrite some code connecting to ACL and replace HashSet with prefix set to improve ACL fi

Re: [PR] MINOR: Remove unused `ApiVersions` variable from Sender and RecordAccumulator [kafka]

2025-04-07 Thread via GitHub
Yunyung commented on PR #19399: URL: https://github.com/apache/kafka/pull/19399#issuecomment-2782382998 > It looks like `apiVersions` in `RecordAccumulator` can also be cleanup. Could you include it in this PR? Thanks. Done. Thanks. -- This is an automated message from the Apache G

Re: [PR] KAFKA-15370: ACL changes to support 2PC (KIP-939) [kafka]

2025-04-07 Thread via GitHub
FrankYang0529 commented on code in PR #19364: URL: https://github.com/apache/kafka/pull/19364#discussion_r2030671804 ## core/src/main/scala/kafka/server/KafkaApis.scala: ## @@ -1520,6 +1520,10 @@ class KafkaApis(val requestChannel: RequestChannel, requestHelper.sendErro

  1   2   >