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

2025-09-22 Thread Jun Rao
which > brokers are followers for which partitions and then group partitions that > share similar followers into the same combined log segment. This could > drastically reduce the metadata fan-out. For example, consider a 4-node > cluster with a replication factor of 2 (RF=2) and two topics,

Re: Apache Kafka 3.9.2 - Release Manager

2025-09-22 Thread Jun Rao
Hi, Chia-Ping, It might be useful to include https://issues.apache.org/jira/browse/KAFKA-19390 in 3.9.2. I have asked the contributor about backporting. Thanks, Jun On Thu, Sep 11, 2025 at 5:36 PM Chia-Ping Tsai wrote: > Yes, we need to ensure that 3.9.2 includes enough bug fixes or patches.

Re: [DISCUSS] KIP-1170: Unify cluster metadata bootstrapping

2025-09-20 Thread Jun Rao
Hi, Jose, Thanks for the reply. JR2. The approach described sounds good to me. Could you include it in the KIP? Jun On Thu, Sep 4, 2025 at 12:17 PM José Armando García Sancio wrote: > Hi Jun, thanks for the feedback. > > On Thu, Aug 28, 2025 at 3:03 PM Jun Rao wrote: > &g

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

2025-09-15 Thread Jun Rao
> cluster. > > To manage the fan-out and reduce overhead, we break down larger metadata > messages into smaller ones. For a combined batch of 10 partitions, we > publish a single combined metadata message and 10 individual messages for > each partition. The combined message is routed

Re: [DISCUSS] KIP-1150 Diskless Topics

2025-09-08 Thread Jun Rao
y > already > > built into classic topics. > > However, local segments are now considered cache elements, do not need to > > be durably stored, and can be built without contacting any other > replicas. > > > > 3. The design has been simplified substantially, by

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

2025-09-05 Thread Jun Rao
needs to retain the metadata before > the active log segment becomes historical segment and uploaded to the > normal tiered storage. The other factor to reduce the metadata size is we > are combining log segments from multiple topic partitions when we upload > onto S3E1Z. > > Tha

Re: [DISCUSS] KIP-1170: Unify cluster metadata bootstrapping

2025-08-28 Thread Jun Rao
Hi, Jose, Thanks for the KIP. JR1. "if the snapshot id has an epoch of 0 and a base offset of 0" : I guess that snapshot here means checkpoint? JR2. There seems to be an existing bootstrapping issue related to the metadata version. KRaft client needs to issue ApiVersionRequest during initializat

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

2025-08-26 Thread Jun Rao
into implementation details, but we are open to expanding > the interface to have a cleaner way of supporting this (e.g., only return > metadata, directly update replicated offset). > > Thanks, > Tom & Henry > > On Wed, Aug 6, 2025 at 9:58 AM Jun Rao wrote: > > > Hi, H

[jira] [Created] (KAFKA-19634) document the encoding of nullable struct

2025-08-21 Thread Jun Rao (Jira)
Jun Rao created KAFKA-19634: --- Summary: document the encoding of nullable struct Key: KAFKA-19634 URL: https://issues.apache.org/jira/browse/KAFKA-19634 Project: Kafka Issue Type: Improvement

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

2025-08-21 Thread Jun Rao
y` is valid > only for the controller. As Jun mentioned, the broker encounters an error > in this case. > > (By the way, the table in KIP-1202 has an incorrect value — null is > acceptable for both cases.) > > > > If we want to address `advertised.listeners` in this KIP,

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

2025-08-20 Thread Jun Rao
s > validator should be updated to anyNonDuplicateValues(isNullAllowed = > false, isEmptyAllowed = false). > > 2. The type of advertised.listeners should be changed from string to list. > As for its validator, > I think we can continue the discussion in KIP-1202. > > Best Regar

Re: [DISCUSS] KIP-1147: Improve consistency of command-line arguments

2025-08-19 Thread Jun Rao
for the comment. It's a good point. I've updated the KIP. > > > > Thanks, > > Andrew > > > > > > From: Jun Rao > > Sent: 18 August 2025 17:41 > > To: dev@kafka.apache.org > > Subject: Re: [DISCUSS] KIP-1147: Imp

Re: [DISCUSS] KIP-1147: Improve consistency of command-line arguments

2025-08-18 Thread Jun Rao
t; > Thanks for the reply. > > > > I've made the updates to the KIP. > > > > Thanks, > > Andrew > > > > From: Jun Rao > > Sent: 15 August 2025 18:23 > > To: dev@kafka.apache.org > > Subject:

Re: [DISCUSS] KIP-1147: Improve consistency of command-line arguments

2025-08-15 Thread Jun Rao
e KIP proposes --command-config. > > Was this the kind of thing you were thinking? Do you think > --command-property is worth the change? How about > --formatter-property? Or maybe something else? > > Thanks, > Andrew > > From: Jun R

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

2025-08-14 Thread Jun Rao
Hi, Jiunn-Yang, Thanks for your patience in addressing all the comments. +1 on the KIP. Jun On Thu, Aug 14, 2025 at 5:12 AM Chia-Ping Tsai wrote: > +1 (binding) > > On 2025/08/14 10:07:50 黃竣陽 wrote: > > Hello all, > > > > I believe the KIP discussion is complete, so I’m manually bumping this >

Re: [DISCUSS] KIP-1147: Improve consistency of command-line arguments

2025-08-13 Thread Jun Rao
Hi, Andrew, Thanks for the KIP. If we are replacing --consumer.config with --command-config, should we do the same for --consumer-property for consistency? Jun On Wed, Aug 13, 2025 at 9:21 AM Andrew Schofield < andrew_schofield_j...@outlook.com> wrote: > Hi Jiunn, > Thanks for your comment. >

Re: [DISCUSS] The Path Forward for Saving Cross-AZ Costs KIPs

2025-08-13 Thread Jun Rao
o the effects > it has on latency. > > Also for KIP-1176 you state that the current availability story is weak. I > don't follow, can you please elaborate? > > Regards, > > From: Jun Rao > Sent: Wednesday, August 6, 2025 19:05 > To:

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

2025-08-13 Thread Jun Rao
Hi, Jiunn-Yang, Thanks for the updated KIP. Looks good to me. Jun On Wed, Aug 13, 2025 at 3:08 AM 黃竣陽 wrote: > Hello Jun, > > Thanks for the reply. > > I have updated the KIP according there comments. > > Best Regards, > Jiunn-Yang > > > Jun Rao 於 2025年8月13日

[jira] [Created] (KAFKA-19601) always disallow min.insync.replicas at the broker level

2025-08-12 Thread Jun Rao (Jira)
Jun Rao created KAFKA-19601: --- Summary: always disallow min.insync.replicas at the broker level Key: KAFKA-19601 URL: https://issues.apache.org/jira/browse/KAFKA-19601 Project: Kafka Issue Type

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

2025-08-12 Thread Jun Rao
Hi, Jiunn-Yang, Thanks for the reply. A few more comments. JR50. It seems that you changed the default value for config.providers incorrectly. The change is meant for bootstrap.servers. JR51. Could you document the current behavior if bootstrap.servers is empty in ConsumerConfig, WorkerConfig, P

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

2025-08-12 Thread Jun Rao
y insight is that TTL configuration depends on the business > requirements and usage patterns rather than just the Kafka retention > policy. The PayloadStore implementation should provide flexibility for > users to make this trade-off consciously. >

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

2025-08-11 Thread Jun Rao
if it is good > enough for max.request.size it might be good enough for this. I updated the > KIP anyway to simplify it and keep it to the original size. > > Hope the final version addressed all feedbacks and we can resume with the > voting > > Thanks > Omnia > > >

Re: [DISCUSS] KIP-1030: Change constraints and default values for various configurations

2025-08-11 Thread Jun Rao
Hi, Chia-Ping, It seems that the main issue with frequent segment rolling is the increased open file descriptors. Rolls bue to retention.ms doesn't have this issue since the rolled segment will be deleted immediately. So, we can leave the implementation of retention.ms as it is. Thanks, Jun O

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

2025-08-11 Thread Jun Rao
configuration has its own validator, so we should leave it unchanged. > > [1] > https://github.com/apache/kafka/blob/18045c6ac30921503deffbef1744bb365dc599fb/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/HeaderFromTest.java#L240 > > Best Regards, > Jiunn-Yang > > > 黃竣陽 於 2025年8月6日 晚

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

2025-08-08 Thread Jun Rao
; >>> JR7. PayloadResponse: Why do we have both responseCode and > >>> PayloadStoreException? > >> We can do without responseCode, the initial though was to report > response > >> code form payload store. > >> Update the KIP. > >>>

Re: [DISCUSS] The Path Forward for Saving Cross-AZ Costs KIPs

2025-08-06 Thread Jun Rao
Hi, Luke, Thanks for starting the discussion. I took a look at all three proposals and the following is my assessment. KIP-1150 (diskless): Pros: * has the most benefits to the users. -- most complete saving of cross zone network cost (enabled by leader less design) -- better durability (by lever

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

2025-08-06 Thread Jun Rao
Hi, Henry, Thanks for the KIP. A few comments. JR1. "In the proposed design, we propose using 1 leader in AZ1, 1 follower in AZ2 and 1 S3E1Z bucket in AZ1." This means that if S3E1Z in AZ1 is not available, we lose the availability of that partition, right? JR2. Regarding combined log segments i

Re: [DISCUSS] KIP-1183: Unified Shared Storage

2025-08-06 Thread Jun Rao
Hi, Xinyu, Thanks for the KIP. A few high level comments. JR1. One potential downside of using RF=1 is availability. Since there is no active standby replica, on a failover, the new leader needs to recover the log and rebuild/reload the state before it can serve writes. So, the window of unavaila

Re: [DISCUSS] KIP-1030: Change constraints and default values for various configurations

2025-08-06 Thread Jun Rao
Hi, Divij, Another comment regarding the changes to segment.ms. Currently, retention.ms has the following doc. "Additionally, retention.ms configuration operates independently of " segment.ms" and "segment.bytes" configurations. Moreover, it triggers the rolling of new segment if the retention.ms

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

2025-08-06 Thread Jun Rao
g Tsai 於 2025年8月6日 凌晨12:54 寫道: > > > > > > hi Jiunn-Yang > > > > > > chia05: > > > > > > `task.assigned.groups` and `task.assigned.partitions` should not have > > > default value, right? > > > > > > chia06: > > > > > >

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

2025-08-05 Thread Jun Rao
t; JR46: I have clarified the behavior for null and empty values, so users > can > understand the reasoning behind disallowing empty lists. > > JR48: I've updated the KIP accordingly. > > Best Regards, > Jiunn-Yang > > > Jun Rao 於 2025年8月5日 凌晨12:45 寫道: &g

Re: [DISCUSS] KIP-1066: Mechanism to cordon brokers and log directories

2025-08-04 Thread Jun Rao
switched it to a > tagged field instead. > > Thanks, > Mickael > > > On Mon, Jul 21, 2025 at 11:32 PM Jun Rao wrote: > > > > Hi, Mickael, > > > > Thanks for the reply. Just one more comment. > > > > Why is CordonedLogDirs a tagged field in Br

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

2025-08-04 Thread Jun Rao
al behavior. Therefore, I prefer not to change > this default. > > Best Regards, > Jiunn-Yang > > > Jun Rao 於 2025年8月2日 凌晨2:38 寫道: > > > > Hi, Jiunn-Yang, > > > > Thanks for the reply. A few more comments. > > > > JR42. [6] [7] are not

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

2025-08-01 Thread Jun Rao
the KIP based on your comments. Please take a > look. > > Best Regards, > Jiunn-Yang > > > Jun Rao 於 2025年7月24日 凌晨2:56 寫道: > > > > Hi, Jiunn-Yang, > > > > Thanks for the reply. > > > > JR40. In the table, there are a few configs for which w

Re: [VOTE] 4.1.0 RC1

2025-08-01 Thread Jun Rao
. > > Paolo > > On Thu, 31 Jul 2025 at 22:31, Jun Rao wrote: > > > Hi, Paolo, > > > > Thanks for the context. Do you feel that the doc change addresses your > > concern? > > > > Jun > > > > On Thu, Jul 31, 2025 at 5:13 AM Paolo Patierno

Re: [VOTE] 4.1.0 RC1

2025-07-31 Thread Jun Rao
-operator/issues/11685 > when found out about this behaviour ;-) > > Thanks, > Paolo > > On Wed, 30 Jul 2025 at 18:55, Jun Rao wrote: > > > Hi, Paolo, > > > > There is a PR https://github.com/apache/kafka/pull/20268 for the doc > > change > > for E

Re: [VOTE] 4.1.0 RC1

2025-07-30 Thread Jun Rao
Hi, Paolo, There is a PR https://github.com/apache/kafka/pull/20268 for the doc change for ELR. Hi, Christo, Group rebalance is designed so that each partition is only consumed by a single consumer instance at a time. Share consumer by design allows a single partition to be consumed by more than

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

2025-07-28 Thread Jun Rao
t; JR9. Why do we need the protected field > PayloadStoreException.isRetryable? > Initial thought here was the serializer can retry the upload. But I have > removed all the retry logic from serializer and it will be up to the > PayloadStore provider to implement this if they need it. >

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

2025-07-23 Thread Jun Rao
ig is optional and allows both null and > empty lists. > log.dirs and plugin.path: These are also optional, but only allow > null—empty lists are not permitted. > > Best Regards, > Jiunn-Yang > > > Jun Rao 於 2025年7月23日 凌晨2:28 寫道: > > > > Hi, Jiunn-Yang and Chia-Ping

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

2025-07-22 Thread Jun Rao
Hi, Omnia, Thanks for the KIP. A few comments. JR1. While the KIP is potentially useful, I am wondering who is responsible for retention for the objects in the payload store. Once a message with a reference is deleted, the key of the external object is lost and the object may never be deleted. J

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

2025-07-22 Thread Jun Rao
s no `MirrorSourceTask` can be created. > > > > By contrast, if any `MirrorSourceTask` is created, the configs used by > > `MirrorSourceTask` must contain `task.assigned.partitions` with a > non-empty > > value. > > > > Jun Rao 於 2025年7月22日 週二 上午5:14寫道: > > >

Re: [DISCUSS] KIP-1150 Diskless Topics

2025-07-21 Thread Jun Rao
Hi, Jan, For me, the main gap of KIP-1150 is the support of all existing client APIs. Currently, there is no design for supporting APIs like transactions and queues. Thanks, Jun On Mon, Jul 21, 2025 at 3:53 AM Jan Siekierski wrote: > Would it be a good time to ask for the current status of th

Re: [DISCUSS] KIP-1066: Mechanism to cordon brokers and log directories

2025-07-21 Thread Jun Rao
anges but upon > reopening the page on the wiki, my changes appeared as draft. > I saved again and verified the page has updated now. > > Thanks, > Mickael > > On Fri, Jul 18, 2025 at 5:52 PM Jun Rao wrote: > > > > Hi, Michael, > > > > Thanks for the rep

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

2025-07-21 Thread Jun Rao
Hi, Jiunn-Yang and Chia-Ping, Thanks for the reply. JR34. Hmm, it seems that the code in taskConfigs() could return an empty list for a task if knownSourceTopicPartitions is less than maxTasks, Chia-Ping? JR36. Sounds good. Could you update the KIP? Also, should we do the same for ssl.cipher.sui

Re: [DISCUSS] KIP-1066: Mechanism to cordon brokers and log directories

2025-07-18 Thread Jun Rao
tly mentioned it in the KIP now. > > JR12: Right, as the description says it's a list! I've adjusted the type. > > Thanks, > Mickael > > > On Wed, Jul 16, 2025 at 8:43 PM Jun Rao wrote: > > > > Hi, Mickael, > > > > Thanks for the updated

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

2025-07-17 Thread Jun Rao
t means > > consumer has nothing to seek/poll. > > > > chia02. task.assigned.groups has analogous issue. > > > > chia03. MirrorCheckpointConfig's `groups` and `groups.exclude` should not > > accept null, since they are complied by regular expressi

Re: [DISCUSS] KIP-1066: Mechanism to cordon brokers and log directories

2025-07-16 Thread Jun Rao
ould un-cordon the > brokers to > > > allow normal placements to occur. Since we expect the operator to > > > eventually come back and un-cordon the brokers, this means it will > > > occasionally be forgotten (since it's a human task). WDYT about a new > > &g

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

2025-07-16 Thread Jun Rao
ation.algorithms: > defaultVerificationAlgorithms(crypto) → ConfigDef.LambdaValidator > > RestServerConfig > admin.listeners: null → AdminListenersValidator > listeners: List.of("http://:8083";) → ListenersValidator > > Best Regards, > Jiunn-Yang > > > Jun Rao 於 202

Re: [VOTE] 4.1.0 RC0

2025-07-16 Thread Jun Rao
gt; > Mickael > > > > > > > > On Mon, Jul 14, 2025 at 8:42 PM Bill Bejeck > wrote: > > > > > > > > > > Hi Mickael, > > > > > > > > > > I've found another blocker as well KAFKA-19504 >

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

2025-07-15 Thread Jun Rao
] > > > https://github.com/apache/kafka/blob/trunk/server/src/main/java/org/apache/kafka/server/config/AbstractKafkaConfig.java#L133 > > [1] > > > https://github.com/apache/kafka/blob/7ea32a0e938c22119f11908aa419aaf0ffd9b6d8/core/src/main/scala/kafka/server/KafkaConfig.scala#

[jira] [Created] (KAFKA-19512) add missing MVs in MetadataVersionTest

2025-07-15 Thread Jun Rao (Jira)
Jun Rao created KAFKA-19512: --- Summary: add missing MVs in MetadataVersionTest Key: KAFKA-19512 URL: https://issues.apache.org/jira/browse/KAFKA-19512 Project: Kafka Issue Type: Improvement

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

2025-07-14 Thread Jun Rao
; > `sasl.oauthbearer.expected.audience`, `ssl.cipher.suites`, > > `ssl.enabled.protocols`, > > and `log.dirs` > > > > Jun Rao 於 2025年7月11日 週五 上午6:59寫道: > > > >> Hi, Chia-Ping, Jiunn-Yang, > >> > >> Thanks for the reply. > >> >

[jira] [Resolved] (KAFKA-19451) Fix flaky test: RemoteIndexCacheTest.testCacheEntryIsDeletedOnRemoval()

2025-07-14 Thread Jun Rao (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-19451?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao resolved KAFKA-19451. - Fix Version/s: 4.2.0 Resolution: Fixed Merged the PR to trunk. > Fix flaky t

Re: [VOTE] 4.1.0 RC0

2025-07-14 Thread Jun Rao
Hi, Paolo, Thanks for identifying the issue. We now have a PR ( https://github.com/apache/kafka/pull/20137) to bump up the latest production metadata version. Jun On Mon, Jul 14, 2025 at 1:58 AM Paolo Patierno wrote: > Hi, > I think I found something which could be a blocker for the release. >

[jira] [Resolved] (KAFKA-19452) Fix flaky test: LogRecoveryTest.testHWCheckpointWithFailuresMultipleLogSegments

2025-07-14 Thread Jun Rao (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-19452?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao resolved KAFKA-19452. - Fix Version/s: 4.2.0 Resolution: Fixed merged the PR to trunk > Fix flaky t

Re: [VOTE] KIP-932: Queues for Kafka

2025-07-11 Thread Jun Rao
apshot records so it is possible to manage > pruning of records correctly. > * Adjusted some of the config defaults and limits such as maximum for lock > duration which didn't make sense. > > KIP-932 mentions some future work and I have started to create new KIPs > for this.

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

2025-07-10 Thread Jun Rao
pears to signify that no > cipher suites are accepted. However, in the codebase, an empty list is > handled as if all available cipher suites are supported [0]. We should not > support the case of "no supported cipher suite," as it doesn't make sense. > > > > In sho

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

2025-07-08 Thread Jun Rao
". Instead, we should use an empty list as the default > value. > > > For example, `sasl.oauthbearer.expected.audience` should never has > "null" > > > value. That could eliminate the null check. > > > > > > For another, there is an additional is

Re: [VOTE] KIP-1186: Update AddRaftVoterRequest RPC to support auto-join

2025-07-08 Thread Jun Rao
Hi, Kevin, Thanks for the KIP. +1 Jun On Wed, Jun 25, 2025 at 7:54 AM Kevin Wu wrote: > Hello all, > > I am manually bumping this thread. > Any feedback or votes would be appreciated. > > Best regards, > Kevin Wu > > On Tue, Jun 17, 2025 at 9:55 AM Kevin Wu wrote: > > > Hello all, > > > > I w

Re: [DISCUSS] KIP-1186: Update AddRaftVoterRequest RPC to support auto-join

2025-07-08 Thread Jun Rao
Hi, Kevin, Thanks for the explanation. The KIP looks good to me now. Jun On Tue, Jul 8, 2025 at 9:07 AM Kevin Wu wrote: > Hi Jun, > > > So, the new controller > > should be able to send a version of the AddRaftVoter request that the > > leader supports, right? > > The new controller can send a

[jira] [Resolved] (KAFKA-7442) forceUnmap mmap on linux when index resize

2025-07-08 Thread Jun Rao (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-7442?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao resolved KAFKA-7442. Fix Version/s: 4.2.0 Resolution: Duplicate This is not fixed in https://issues.apache.org/jira

[jira] [Resolved] (KAFKA-19390) AbstractIndex#resize() does not release old mmap on Linux

2025-07-08 Thread Jun Rao (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-19390?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao resolved KAFKA-19390. - Fix Version/s: 4.2.0 Resolution: Fixed merged the PR to trunk > AbstractIndex#resize() does

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

2025-07-07 Thread Jun Rao
specify > whether a configuration permits null values or not. > > JR13: I have updated the table > > > > Jun Rao 於 2025年7月1日 清晨6:30 寫道: > > > > Hi, Jiunn-Yang, > > > > Sorry for the late reply. A few more comments. > > > > JR10. The doc for

[jira] [Created] (KAFKA-19466) LogConcurrencyTest should close the log when the test completes

2025-07-02 Thread Jun Rao (Jira)
Jun Rao created KAFKA-19466: --- Summary: LogConcurrencyTest should close the log when the test completes Key: KAFKA-19466 URL: https://issues.apache.org/jira/browse/KAFKA-19466 Project: Kafka Issue

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

2025-06-30 Thread Jun Rao
ed the configurations that will > > be validated using this logic. Please feel free to share any feedback or > suggestions. > > > > Best Regards, > > Jiunn-Yang > > > >> Jun Rao 於 2025年5月23日 凌晨2:29 寫道: > >> > >> Hi, Jiunn-Yang, > >&g

[jira] [Created] (KAFKA-19452) Fix flaky test: LogRecoveryTest.testHWCheckpointWithFailuresMultipleLogSegments

2025-06-30 Thread Jun Rao (Jira)
Jun Rao created KAFKA-19452: --- Summary: Fix flaky test: LogRecoveryTest.testHWCheckpointWithFailuresMultipleLogSegments Key: KAFKA-19452 URL: https://issues.apache.org/jira/browse/KAFKA-19452 Project: Kafka

[jira] [Created] (KAFKA-19451) Fix flaky test: RemoteIndexCacheTest.testCacheEntryIsDeletedOnRemoval()

2025-06-30 Thread Jun Rao (Jira)
Jun Rao created KAFKA-19451: --- Summary: Fix flaky test: RemoteIndexCacheTest.testCacheEntryIsDeletedOnRemoval() Key: KAFKA-19451 URL: https://issues.apache.org/jira/browse/KAFKA-19451 Project: Kafka

Re: [DISCUSS] KIP-1186: Update AddRaftVoterRequest RPC to support auto-join

2025-06-27 Thread Jun Rao
Hi, Kevin, Thanks for the KIP. Overall, it looks reasonable to me. Regarding "The follower should not send the AddRaftVoter request if the leader doesn't support the version, because we do not want to cause the unavailability scenario described above. Therefore, the new field should not be ignore

[jira] [Resolved] (KAFKA-19325) improve forceComplete() in DelayedOperation

2025-06-10 Thread Jun Rao (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-19325?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao resolved KAFKA-19325. - Resolution: Won't Fix It seems that changing forceComplete() to return void makes the usage

[jira] [Resolved] (KAFKA-19326) flaky test GroupMetadataManagerTest.testShareGroupInitializeSuccess()

2025-05-23 Thread Jun Rao (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-19326?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao resolved KAFKA-19326. - Fix Version/s: 4.1.0 Resolution: Fixed It seems this is fixed in [https://github.com/apache

[jira] [Created] (KAFKA-19326) flaky test GroupMetadataManagerTest.testShareGroupInitializeSuccess()

2025-05-23 Thread Jun Rao (Jira)
Jun Rao created KAFKA-19326: --- Summary: flaky test GroupMetadataManagerTest.testShareGroupInitializeSuccess() Key: KAFKA-19326 URL: https://issues.apache.org/jira/browse/KAFKA-19326 Project: Kafka

[jira] [Created] (KAFKA-19325) improve forceComplete() in DelayedOperation

2025-05-23 Thread Jun Rao (Jira)
Jun Rao created KAFKA-19325: --- Summary: improve forceComplete() in DelayedOperation Key: KAFKA-19325 URL: https://issues.apache.org/jira/browse/KAFKA-19325 Project: Kafka Issue Type: Improvement

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

2025-05-22 Thread Jun Rao
Hi, Jiunn-Yang, It seems there are quite a few other configs of type list (e.g. several SSL related ones). It would be useful to understand if empty lists are valid for them or not. Thanks, Jun On Tue, May 13, 2025 at 10:12 AM Jun Rao wrote: > Hi, Luke, > > Thanks for the reply

Re: [DISCUSS] KIP-1150 Diskless Topics

2025-05-16 Thread Jun Rao
ited by a single socket connection. We could follow up on this in the sub KIPs. Thanks, Jun On Fri, May 16, 2025 at 6:18 AM Ivan Yurchenko wrote: > > On Tue, May 13, 2025, at 19:34, Jun Rao wrote: > > > > JR4. "Balance traffic among brokers and eliminate broker hotspot

Re: [DISCUSS] KIP-1140: Avoid to return null value in Map from public api of consumer

2025-05-15 Thread Jun Rao
challenging, and we currently have > many offset-related APIs that use null values. Therefore, it would be > acceptable to maintain the current pattern. > > Additionally, I've opened KAFKA-19284 to add documentation regarding the > null value in all offset-related results. > &

Re: [DISCUSS] KIP-1140: Avoid to return null value in Map from public api of consumer

2025-05-14 Thread Jun Rao
> > > > [0] > > > https://github.com/akka/alpakka-kafka/blob/61cae8c04eff05f1408de8d7d0080a0be4c00aac/core/src/main/scala/akka/kafka/internal/KafkaConsumerActor.scala#L365 > > [1] > > > https://github.com/zio/zio-kafka/blob/adfff39bdd86b75f6035bdfb90935f7800d30b52/zio-kafka/src/m

[jira] [Resolved] (KAFKA-19234) broker should return UNAUTHORIZATION error for non-existing topic in produce request

2025-05-14 Thread Jun Rao (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-19234?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao resolved KAFKA-19234. - Fix Version/s: 4.1.0 Resolution: Fixed merged the PR to trunk > broker should ret

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

2025-05-13 Thread Jun Rao
Hi, Kevin, Thanks for the reply. 4. There seems to be some inconsistency. For MinimumSupportedLevel, we have two metrics, one with the package kafka.controller and another with kafka.server. For FinalizedLevel, we only have one metric with the package kafka.server. Could we choose a more consiste

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

2025-05-13 Thread Jun Rao
retention configurations instead. > > > > > We could also consider supporting an empty cleanup.policy by fixing the > > > issue in remote storage. But then the user may never realize this if > > it's a > > > mistake. > > > > Good catch! The &q

Re: [DISCUSS] KIP-1150 Diskless Topics

2025-05-13 Thread Jun Rao
Hi, Josep, Thanks for the KIP. At the highlevel, the KIP is well thought through and provides multiple benefits for Kafka in the Cloud. A few comments below. JR1. One of the key motivations is to eliminate inter-zone data transfer costs from Kafka replication. It would be useful to provide a shor

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

2025-05-12 Thread Jun Rao
Hi, Kevin, Thanks for the updated KIP. A couple of more comments. 4. Should we expose the FinalizedLevel metric on the controller too? 5. Some of the metric names are camel case and some others use dash. It would be useful to be consistent. Jun On Mon, May 12, 2025 at 7:06 AM Kevin Wu wrote:

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

2025-05-12 Thread Jun Rao
without any automatic cleanup." > This is the motivation we mentioned in the KIP, but to me, backward > compatibility is much more important than "a clear and direct way to config > kafka". > Do we really have to change the "cleanup.policy" config? > > T

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

2025-05-07 Thread Jun Rao
Hi, Kevin, Thanks for the KIP. A few comments below. 1. For completeness, there is another feature streams.version. 2. "The default value of this metric should be the default value of the feature." This metric reports the finalized level for each feature, right? If a feature is not set, should t

[jira] [Resolved] (KAFKA-19212) Fix ElectionWasClean bug when partition reassignment completes

2025-05-07 Thread Jun Rao (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-19212?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao resolved KAFKA-19212. - Resolution: Fixed merged the PR to trunk > Fix ElectionWasClean bug when partition reassignm

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

2025-05-06 Thread Jun Rao
d to prevent users pass empty config, so should it > > be non-empty too? > > > > For example, `inNonEmpty()` should throw exception directly. > > > > Best, > > Chia-Ping > > > > Jun Rao 於 2025年5月6日 週二 上午12:28寫道: > > > >> Hi, Jiunn-Yang, &g

Re: [VOTE] KIP-1139: Add support for OAuth jwt-bearer grant type

2025-05-05 Thread Jun Rao
Hi, Kirk, Thanks for the KIP. +1 Jun On Fri, May 2, 2025 at 9:15 AM Kirk True wrote: > Hi, > > Thanks to all who provided feedback on KIP-1139: Add support for OAuth > jwt-bearer grant type: > > https://cwiki.apache.org/confluence/x/uIxEF > > I would like to call for a vote for inclusion in th

Re: [DISCUSS] KIP-1140: Avoid to return null value in Map from public api of consumer

2025-05-05 Thread Jun Rao
o make a change, it would be useful to make it consistent too. > > > > yes, both consumers have same behavior. > > > > For another, `listStreamsGroupOffsets` and `listShareGroupOffsets` also > add > > null value. They are not in production, but maybe we should

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

2025-05-05 Thread Jun Rao
Hi, Jose, Thanks for the KIP. +1 Jun On Fri, May 2, 2025 at 8:42 AM José Armando García Sancio wrote: > Hi all, > > Thanks for the discussion. I would like to start the voting thread for > KIP-1166: Improve high-watermark replication. If you have additional > comments or questions please use t

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

2025-05-05 Thread Jun Rao
Hi, Jose, Thanks for updating the KIP. It looks good to me now. Jun On Mon, May 5, 2025 at 9:42 AM José Armando García Sancio wrote: > Hi Jun, > > On Mon, May 5, 2025 at 12:22 PM Jun Rao wrote: > > Since MV is part of the public interface, it would be useful to call it >

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

2025-05-05 Thread Jun Rao
uot;, could you add > > description to remind readers that "clean.policy=" should be replaced by > > "clean.policy=none" if they do want to disable delete and compact. > > > > Best, > > Chia-Ping > > > > Jun Rao 於 2025年5月3日 週六 上午1:55寫

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

2025-05-05 Thread Jun Rao
, > > On Fri, May 2, 2025 at 6:57 PM Jun Rao wrote: > > Thanks for the reply. You mentioned that we still need an MV, but it's no > > longer in the KIP. > > Thanks. I mentioned the new MV in the "Sending" section of the KIP. > > -- > -José >

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

2025-05-02 Thread Jun Rao
Hi, Jose, Thanks for the reply. You mentioned that we still need an MV, but it's no longer in the KIP. Jun On Fri, May 2, 2025 at 12:28 PM José Armando García Sancio wrote: > Hi Jun, > > On Fri, May 2, 2025 at 1:47 PM Jun Rao wrote: > > Since we are only doing the fast

[jira] [Created] (KAFKA-19234) broker should return UNAUTHORIZATION error for non-existing topic in produce request

2025-05-02 Thread Jun Rao (Jira)
Jun Rao created KAFKA-19234: --- Summary: broker should return UNAUTHORIZATION error for non-existing topic in produce request Key: KAFKA-19234 URL: https://issues.apache.org/jira/browse/KAFKA-19234 Project

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

2025-05-02 Thread Jun Rao
> > if cleanup.policy is empty, the broker will fail to start. > > I’ve updated the KIP accordingly. > > Best Regards, > Jiunn-Yang > > > Jun Rao 於 2025年5月2日 凌晨12:50 寫道: > > > > Hi, Jiunn-Yang, > > > > Thanks for the reply. Sounds good.

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

2025-05-02 Thread Jun Rao
probably don't need to set the HWM field. Jun On Fri, May 2, 2025 at 7:48 AM José Armando García Sancio wrote: > Hi Jun, > > On Thu, May 1, 2025 at 12:38 PM Jun Rao wrote: > > We could probably just keep the new HighWatermark field as described in > the > > KIP, but

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

2025-05-01 Thread Jun Rao
e 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-Y

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

2025-05-01 Thread Jun Rao
arcía Sancio wrote: > 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. >

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

2025-04-30 Thread Jun Rao
Hi, Ismael, Thanks for the pointers. Hi, Jose, KIP-392 originally considered propagating HWM to followers. If there is a performance concern, we could focus only on the KRaft metadata topic. Also, KIP-392 implemented HWM propagation without adding the HWM field in the fetch request. Instead, the

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

2025-04-30 Thread Jun Rao
ion phase > instead of relying on the current behavior.” > > This reflects the correct behavior. > > Best Regards, > Jiunn-Yang > > > Jun Rao 於 2025年4月25日 凌晨1:17 寫道: > > > > Hi, Jiunn-Yang, > > > > "The main behavioral difference introduced

Re: [DISCUSS] KIP-1140: Avoid to return null value in Map from public api of consumer

2025-04-30 Thread Jun Rao
this is a bad idea, so let's not > >consider it. > > > > By the way, DescribeProducersResult#partitionResult(final TopicPartition > > partition) needs to be considered as well. > > Best, > > Chia-Ping > > > > Jun Rao 於 2025年4月30日 週三 上午6:30寫道: > &

  1   2   3   4   5   6   7   8   9   10   >