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

2025-05-08 Thread Luke Chen
Hi Xinyu and Henry, I think the WAL metadata in KIP1176 is not for log recover, the log recovery still loads log segments locally. The WAL metadata is for leader <-> follower information sharing only. Is my understanding correct? About the WAL metadata, as I mentioned earlier, I still worry about

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

2025-05-08 Thread Luke Chen
Hi Henry, Some more questions: 2. It seems in the KIP, I _assume_ the fast cloud storage will always be available without issue. What will happen when the fast cloud storage is down? Do we fail the producer write immediately? Or do we have any fall back mechanism? Maybe we can fall back to the tra

[DISCUSS] KIP-1178: Introduce remote.max.partition.fetch.bytes in Consumer

2025-05-08 Thread Kamal Chandraprakash
Hi all, I've opened the KIP-1178 to add a new config 'remote.max.partition.fetch.bytes' in the consumer. This config allows it to read from remote storage faster. https://cwiki.apache.org/confluence/display/KAFKA/KIP-1178%3A+Introduce+remote.max.partition.fetch.bytes+config+in+Consumer Please ta

[DISCUSS] KIP-1175: Fix the typo `PARTITIONER_ADPATIVE_PARTITIONING_ENABLE` in ProducerConfig

2025-05-08 Thread 鍾明諺
Hi all, I would like to start discussions on a trivial KIP https://cwiki.apache.org/confluence/display/KAFKA/KIP-1175%3A+Fix+the+typo+%60PARTITIONER_ADPATIVE_PARTITIONING_ENABLE%60+in+ProducerConfig Best, Ming-Yen

[jira] [Resolved] (KAFKA-19087) Move TransactionState to transaction-coordinator module

2025-05-08 Thread Chia-Ping Tsai (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-19087?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Chia-Ping Tsai resolved KAFKA-19087. Fix Version/s: 4.1.0 Resolution: Fixed > Move TransactionState to transaction-coord

Re: [VOTE] KIP-1157: Enforce KafkaPrincipalSerde Implementation for KafkaPrincipalBuilder

2025-05-08 Thread Chia-Ping Tsai
hi Szu-Yung please update the KIP to include the VOTE thread. thanks, Chia-Ping Jhen-Yung Hsu 於 2025年5月7日 週三 上午3:20寫道: > +1 (non-binding) > > Yunyung > > On 2025/05/05 08:03:12 Johnny Wang wrote: > > Hi all, > > I would like to call a vote for KIP-1157. > > https://cwiki.apache.org/confluence/

Re: [DISCUSS] KIP-1153: Kafka Streams `CloseOptions` should not have a public constructor

2025-05-08 Thread 黃竣陽
Hi all, Thanks for all the feedback. I’ve updated the KIP and introduced a CloseOptionsInternal class that provides a getter method. Best Regards, Jiunn-Yang > Chia-Ping Tsai 於 2025年5月8日 下午1:55 寫道: > >> One more nit: we should remove the getters. There are useless. -- In > Kafka Streams, we f

Re: [DISCUSS] KIP-1175: Fix the typo `PARTITIONER_ADPATIVE_PARTITIONING_ENABLE` in ProducerConfig

2025-05-08 Thread Andrew Schofield
Hi Ming-Yen, Thanks for the KIP. I'm glad to see that this is going to be cleaned up :) AS1: There's no need to deprecate the DOC constant because it's private. Thanks, Andrew From: 鍾明諺 Sent: 08 May 2025 09:21 To: dev@kafka.apache.org Subject: [DISCUSS]

Re: [VOTE] KIP-1157: Enforce KafkaPrincipalSerde Implementation for KafkaPrincipalBuilder

2025-05-08 Thread Johnny Wang
Hi, Chia-Ping Updated, thanks for reminding me. Wang Chia-Ping Tsai 於 2025年5月9日 週五 上午12:55寫道: > hi Szu-Yung > > please update the KIP to include the VOTE thread. > > thanks, > Chia-Ping > > Jhen-Yung Hsu 於 2025年5月7日 週三 上午3:20寫道: > > > +1 (non-binding) > > > > Yunyung > > > > On 2025/05/05 08:

Re: [DISCUSS] KIP-1180: Add a generic feature level metric

2025-05-08 Thread Kevin Wu
Hey Jun, Thanks for the comments. 1. I'll update the KIP. My trunk is a bit stale. 2. Yeah, the metric should report the finalized feature level for the feature. And if it is not set, the metric will report 0. 3. I'll update the KIP with a timeline. Thanks, Kevin On Wed, May 7, 2025 at 3:10 PM K

Re: [DISCUSS] KIP-1178: Introduce remote.max.partition.fetch.bytes in Consumer

2025-05-08 Thread Andrew Schofield
Hi Kamal, Thanks for the KIP. While it makes a lot of sense to me to be able to control the fetching from remote storage to make sure it's sympathetic to the characteristics of the storage provider, it seems to me that extending this concept all the way to the individual consumers is not a good

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

2025-05-08 Thread José Armando García Sancio
Hi all, The KIP is accepted with 4 binding voters (José, Jun, David Arthur and Colin) and one non-binding vote (Alyssa). Thanks everyone that voted and participated in the discussion, -- -José

Re: [PR] KAFKA-19209: Clarify index.interval.bytes impact on offset and time index [kafka-site]

2025-05-08 Thread via GitHub
shmily7829 commented on code in PR #686: URL: https://github.com/apache/kafka-site/pull/686#discussion_r2075855172 ## 40/generated/topic_config.html: ## @@ -111,7 +111,12 @@ index.interval.bytes -This setting controls how frequently Kafka adds an index entry to its offset index

Re: [DISCUSS] KIP-1180: Add a generic feature level metric

2025-05-08 Thread Chia-Ping Tsai
hi Kevin thanks for this KIP. It does offer the useful metrics. One question: Have you considered adding the supported versions to the metrics? Currently, admin#describeFeatures cannot return supported versions for a specific server because we cannot configure the target server handling the RPC r

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

2025-05-08 Thread Henry Haiying Cai
Xinyu, Thanks for your time and comments again.  For the metadata management using a Kafka topic, we are basically following what KIP-405 was doing by reusing the same metadata topic and the same metadata cache based on that topic. It's also doable by scanning the WAL in S3 to get the metadata,

Re: [DISCUSS] KIP-1175: Fix the typo `PARTITIONER_ADPATIVE_PARTITIONING_ENABLE` in ProducerConfig

2025-05-08 Thread Ming-Yen Chung
Hi Andrew, Thanks for pointing that out. I’ve removed the deprecation information regarding the DOC constant from the KIP. Andrew Schofield 於 2025年5月8日 週四 下午9:52寫道: > Hi Ming-Yen, > Thanks for the KIP. I'm glad to see that this is going to be cleaned up :) > > AS1: There's no need to deprecate t

[jira] [Resolved] (KAFKA-19209) "index.interval.bytes" config doc should also mention time index

2025-05-08 Thread Luke Chen (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-19209?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Luke Chen resolved KAFKA-19209. --- Fix Version/s: 4.1.0 Resolution: Fixed > "index.interval.bytes" config doc should also mentio

Re: [VOTE] 3.9.1 RC1

2025-05-08 Thread TengYao Chi
Hi Matthias, Thanks for the testing and vote. Are you saying that when browsing the 3.9.1 documentation, the top-level links still point to the 4.0 QuickStart? Could you please elaborate on the issue in more detail? Best, TengYao Matthias J. Sax 於 2025年5月9日 週五,上午5:57寫道: > Hi, > > I did the foll

New Contributor to Apache Kafka cwiki/Jira

2025-05-08 Thread Peter Corless
Hi folks! Wanted to get access to the Apache Kafka cwiki/JIRA to submit a new KIP, as per the instructions on this page . wiki ID: petercorless Jira ID: petercorless Let me know if this is sufficient to get me going,

Re: [VOTE] 3.9.1 RC1

2025-05-08 Thread Matthias J. Sax
Hi, I did the following: - verify all signatures - build from source (OpenJDK 17, Scala 2.13) - run all tests `OffsetsApiIntegrationTest` failed, but re-run passed - run quick start (core/connect/streams) using Scala 2.12 binaries (KRaft mode) - downloaded both docker images; started them up,

[jira] [Created] (KAFKA-19256) Only send IQ information on assignment changes

2025-05-08 Thread Bill Bejeck (Jira)
Bill Bejeck created KAFKA-19256: --- Summary: Only send IQ information on assignment changes Key: KAFKA-19256 URL: https://issues.apache.org/jira/browse/KAFKA-19256 Project: Kafka Issue Type: Sub-

[jira] [Created] (KAFKA-19255) KRaft request manager should support one in-flight request per request type

2025-05-08 Thread Kevin Wu (Jira)
Kevin Wu created KAFKA-19255: Summary: KRaft request manager should support one in-flight request per request type Key: KAFKA-19255 URL: https://issues.apache.org/jira/browse/KAFKA-19255 Project: Kafka

Re: New Contributor to Apache Kafka cwiki/Jira

2025-05-08 Thread Matthias J. Sax
You should be all set. -Matthias On 5/8/25 2:40 PM, Peter Corless wrote: Hi folks! Wanted to get access to the Apache Kafka cwiki/JIRA to submit a new KIP, as per the instructions on this page . wiki ID: petercorl

Re: [DISCUSS] KIP-1178: Introduce remote.max.partition.fetch.bytes in Consumer

2025-05-08 Thread Kamal Chandraprakash
Hi Andrew, Thanks for the review! The initial idea was to introduce the configuration on the broker side, similar to how remote.fetch.max.wait.ms complements fetch.max.wait.ms: - fetch.max.wait.ms — configured via ConsumerConfig

Re: [DISCUSS] KIP-1180: Add a generic feature level metric

2025-05-08 Thread Kevin Wu
Hey Chia-Ping, I hadn't considered adding the supported versions for each feature as a metric, but I'm not sure if it's helpful for monitoring the progress of an upgrade/downgrade of a feature. For example, if a node doesn't support a particular feature level we're upgrading to, we shouldn't even

Re: [DISCUSS] KIP-1180: Add a generic feature level metric

2025-05-08 Thread Justine Olshan
Hey Kevin, I think Chia Ping was referring to not the time when the feature upgrade is happening, but when a kafka image upgrade is happening for example. We can't upgrade the feature until all brokers support it, so it would be also helpful to see that in real time? Something like that. I don't

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

2025-05-08 Thread Henry Haiying Cai
Luke, Thanks again for detailed comments and suggestions, see my answer below inline with HC> indentation. On Thursday, May 8, 2025 at 12:59:43 AM PDT, Luke Chen wrote: Hi Xinyu and Henry, I think the WAL metadata in KIP1176 is not for log recover, the log recovery still loads log seg

[jira] [Resolved] (KAFKA-18115) Issue loading big files for performance testing

2025-05-08 Thread Jira
[ https://issues.apache.org/jira/browse/KAFKA-18115?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] José Armando García Sancio resolved KAFKA-18115. Resolution: Fixed > Issue loading big files for performance testin

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

2025-05-08 Thread Henry Haiying Cai
Luke, Thanks for your continued suggestions on this KIP, see an my answer below inline with HC> indentation. On Thursday, May 8, 2025 at 02:10:07 AM PDT, Luke Chen wrote: Hi Henry, Some more questions: 2. It seems in the KIP, I _assume_ the fast cloud storage will always be available

Jenkins build is still unstable: Kafka » Kafka PowerPC Daily » test-powerpc #292

2025-05-08 Thread Apache Jenkins Server
See