Re: [DISCUSS] KIP-1159: Large message reference based Serializer

2025-05-01 Thread Kirk True
Hi Omnia, A very interesting KIP! Thanks for the write up and discussion thus far! A few questions: KT1: In PayloadReferenceValue, Is "payloadStoreClass" intended to be a fully-qualified Java class name, or something else? I'm considering the case where a Producer and/or Consumer are using a n

Re: [DISCUSS] KIP-1176: Tiered Storage for Active Log Segment

2025-05-01 Thread Stanislav Kozlovski
Thanks for the awesome proposal! Few questions: 1) can we have more detail around the proposed deployment model in GCP/Azure? As of reading, the KIP seems overly focused on AWS - but the feature will need to have viable support for the other two major clouds, right? In particular it would be n

Re: [DISCUSS] KIP-1176: Tiered Storage for Active Log Segment

2025-05-01 Thread Henry Haiying Cai
See answers below On Thursday, May 1, 2025 at 06:14:34 AM PDT, Stanislav Kozlovski wrote: Thanks for the awesome proposal! Few questions: 1) can we have more detail around the proposed deployment model in GCP/Azure? As of reading, the KIP seems overly focused on AWS - but the feature

Re: [DISCUSS] KIP-1176: Tiered Storage for Active Log Segment

2025-05-01 Thread Henry Haiying Cai
Luke, Thanks for your comments, see my answers below inline. On Thursday, May 1, 2025 at 03:20:54 AM PDT, Luke Chen wrote: Hi Henry, This is a very interesting proposal! I love the idea to minimize the code change to be able to quickly get delivered. Thanks for proposing this! Some qu

Re: [DISCUSS] KIP-1166: Improve high-watermark replication

2025-05-01 Thread Jun Rao
Hi, Jose, Thanks for the reply. We could probably just keep the new HighWatermark field as described in the KIP, but limit it only for KRaft. Also, since HighWatermark is a tagged field, it probably doesn't need to be gated by an MV. Jun On Thu, May 1, 2025 at 7:17 AM José Armando García Sancio

[jira] [Resolved] (KAFKA-18760) Deprecate Optional and return String from public EndPoint#Listener

2025-05-01 Thread PoAn Yang (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-18760?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] PoAn Yang resolved KAFKA-18760. --- Fix Version/s: 4.1.0 Resolution: Fixed > Deprecate Optional and return String from public End

Re: [DISCUSS] KIP-1162: Redesign ClientQuotaCallback#updateClusterMetadata

2025-05-01 Thread Kuan Po Tseng
Hi Chia-Ping, That works too. But in that case, the original method name feels inappropriate. So, as you suggested, I’ll add the topic parameter and rename the method from: Map partitionsForNode(int nodeId) to: Map partitions(String topic, int nodeId) Since the caller already knows the topic n

Re: [DISCUSS] KIP-1176: Tiered Storage for Active Log Segment

2025-05-01 Thread Luke Chen
Hi Henry, This is a very interesting proposal! I love the idea to minimize the code change to be able to quickly get delivered. Thanks for proposing this! Some questions: 1. In this KIP, we add one more tier of storage. That is: local disk -> fast object store -> slow object store. Why can't we a

[jira] [Resolved] (KAFKA-19169) Enhance AuthorizerIntegrationTest for share group APIs

2025-05-01 Thread Andrew Schofield (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-19169?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Andrew Schofield resolved KAFKA-19169. -- Fix Version/s: 4.1.0 Resolution: Fixed > Enhance AuthorizerIntegrationTest for

[jira] [Resolved] (KAFKA-19015) Remove share session from cache on connection drop

2025-05-01 Thread Andrew Schofield (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-19015?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Andrew Schofield resolved KAFKA-19015. -- Fix Version/s: 4.1.0 Resolution: Fixed > Remove share session from cache on con

Re: [DISCUSS] KIP-1166: Improve high-watermark replication

2025-05-01 Thread José Armando García Sancio
Hi Ismael, On Wed, Apr 30, 2025 at 5:19 PM Ismael Juma wrote: > We did indeed run into a perf regression related to increased fetch request > rate due to hw propagation to followers: > > https://issues.apache.org/jira/browse/KAFKA-9731 Thanks for the information. This is helpful. -- -José

[jira] [Created] (KAFKA-19226) Add test_console_share_consumer system test

2025-05-01 Thread Chirag Wadhwa (Jira)
Chirag Wadhwa created KAFKA-19226: - Summary: Add test_console_share_consumer system test Key: KAFKA-19226 URL: https://issues.apache.org/jira/browse/KAFKA-19226 Project: Kafka Issue Type: Sub

[jira] [Resolved] (KAFKA-19210) investigate flakiness in GroupMetadataManagerTest.testShareGroupHeartbeatInitializeOnPartitionUpdate

2025-05-01 Thread Chirag Wadhwa (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-19210?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Chirag Wadhwa resolved KAFKA-19210. --- Resolution: Fixed > investigate flakiness in > GroupMetadataManagerTest.testShareGroupHeart

Re: [DISCUSS] KIP-1161: cleanup.policy shouldn't be empty

2025-05-01 Thread 黃竣陽
Hello Jun, Since ValidList is a public API, we need to maintain backward compatibility. Therefore, the isEmptyAllowed flag is necessary. We can update the signature of isNonEmpty() to remove the boolean parameter. Best Regards, Jiunn-Yang > Jun Rao 於 2025年5月1日 凌晨4:17 寫道: > > Hi, Jiunn-Yang,

Re: [DISCUSS] KIP-1166: Improve high-watermark replication

2025-05-01 Thread José Armando García Sancio
Hi Jun, On Wed, Apr 30, 2025 at 6:28 PM Jun Rao wrote: > Also, KIP-392 > implemented HWM propagation without adding the HWM field in the fetch > request. Instead, the leader remembers the last propagated HWM to a > follower. I thought about this when designing the feature, mainly to avoid having

Re: [DISCUSS] KIP-1161: cleanup.policy shouldn't be empty

2025-05-01 Thread Jun Rao
Hi, Jiunn-Yang, Thanks for the reply. Sounds good. Just a minor comment. It would be useful to document that during upgrade, if cleanup.policy is empty, the broker will fail to start. Jun On Thu, May 1, 2025 at 8:40 AM 黃竣陽 wrote: > Hello Jun, > > Since ValidList is a public API, we need to ma

[jira] [Resolved] (KAFKA-19073) Add Transactional ID pattern filter to ListTransactions API

2025-05-01 Thread Chia-Ping Tsai (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-19073?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Chia-Ping Tsai resolved KAFKA-19073. Fix Version/s: 4.1.0 Resolution: Fixed > Add Transactional ID pattern filter to Lis

[jira] [Created] (KAFKA-19227) Fix performance for share fetch with piggyback acknowledgements

2025-05-01 Thread Apoorv Mittal (Jira)
Apoorv Mittal created KAFKA-19227: - Summary: Fix performance for share fetch with piggyback acknowledgements Key: KAFKA-19227 URL: https://issues.apache.org/jira/browse/KAFKA-19227 Project: Kafka

[jira] [Resolved] (KAFKA-19146) Merge org.apache.kafka.server.common.OffsetAndEpoch and org.apache.kafka.raft.OffsetAndEpoch

2025-05-01 Thread Chia-Ping Tsai (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-19146?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Chia-Ping Tsai resolved KAFKA-19146. Fix Version/s: 4.1.0 Resolution: Fixed > Merge org.apache.kafka.server.common.Offse

[jira] [Created] (KAFKA-19230) Fail build for FunctionalInterfaceMethodChanged warning

2025-05-01 Thread Matthias J. Sax (Jira)
Matthias J. Sax created KAFKA-19230: --- Summary: Fail build for FunctionalInterfaceMethodChanged warning Key: KAFKA-19230 URL: https://issues.apache.org/jira/browse/KAFKA-19230 Project: Kafka

[jira] [Created] (KAFKA-19228) Formatting with `--no-initial-controllers` flag should not write kraft version control record

2025-05-01 Thread Kevin Wu (Jira)
Kevin Wu created KAFKA-19228: Summary: Formatting with `--no-initial-controllers` flag should not write kraft version control record Key: KAFKA-19228 URL: https://issues.apache.org/jira/browse/KAFKA-19228

Re: [VOTE] 3.9.1 RC1

2025-05-01 Thread Luke Chen
Hi all, So far, we've got 3 non-binding votes for v3.9.1 release. Thanks everyone who helped validate this! Could PMC members (and contributors) take some time to help validate this RC build? Thanks. Luke Thanks. Luke On Tue, Apr 29, 2025 at 10:34 AM TengYao Chi wrote: > Hi everyone, > > I w

[jira] [Created] (KAFKA-19229) fix flaky testShareGroupDescribeWithGroupDescribeAndTopicDescribeAcl and testShareGroupDescribeWithoutGroupDescribeAcl

2025-05-01 Thread Chia-Ping Tsai (Jira)
Chia-Ping Tsai created KAFKA-19229: -- Summary: fix flaky testShareGroupDescribeWithGroupDescribeAndTopicDescribeAcl and testShareGroupDescribeWithoutGroupDescribeAcl Key: KAFKA-19229 URL: https://issues.apache.or