Re: KIP-1141: Simplifying MetadataQuorumCommand by Leveraging Admin API for Controller Management

2025-07-31 Thread Chia-Ping Tsai
s for the delayed response. Do you still have any questions > > > regarding the describeConfig part? As Chia-Ping mentioned, we are able > to > > > retrieve all broker configurations through that method. ( > > > > > > https://github.com/apache/kafka/blob/da46cf6e79a

[jira] [Resolved] (KAFKA-19524) connect-plugin-path crashes with UnsupportedOperationException on plugins without ServiceLoader manifests

2025-07-31 Thread Chia-Ping Tsai (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-19524?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Chia-Ping Tsai resolved KAFKA-19524. Fix Version/s: 4.2.0 Resolution: Fixed > connect-plugin-path crashes w

Re: [DISCUSS] KIP-1100: Consider renaming org.apache.kafka.server:type=AssignmentsManager

2025-07-30 Thread Chia-Ping Tsai
me know if you > have any feedback. > > Best Regards, > Jiunn-Yang > > > Chia-Ping Tsai 於 2025年7月18日 凌晨2:58 寫道: > > > > hi Jiunn-Yang > > > > I think this KIP is ready for a VOTE thread. > > > > thanks, > > Chia-Ping > > > >

[jira] [Resolved] (KAFKA-19527) improve the docs of LogDirDescription for remote stroage

2025-07-30 Thread Chia-Ping Tsai (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-19527?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Chia-Ping Tsai resolved KAFKA-19527. Fix Version/s: 4.2.0 Resolution: Fixed > improve the docs of LogDirDescription

[jira] [Created] (KAFKA-19562) Rewrite AbortedTxn by generated protocol

2025-07-29 Thread Chia-Ping Tsai (Jira)
Chia-Ping Tsai created KAFKA-19562: -- Summary: Rewrite AbortedTxn by generated protocol Key: KAFKA-19562 URL: https://issues.apache.org/jira/browse/KAFKA-19562 Project: Kafka Issue Type

[jira] [Resolved] (KAFKA-19545) Remove MetadataVersionValidator

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

[jira] [Created] (KAFKA-19545) Remove MetadataVersionValidator

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

[jira] [Resolved] (KAFKA-17645) KIP-1052: Enable warmup in producer performance test

2025-07-23 Thread Chia-Ping Tsai (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-17645?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Chia-Ping Tsai resolved KAFKA-17645. Fix Version/s: 4.2.0 Resolution: Fixed > KIP-1052: Enable warmup in produ

Re: [VOTE] KIP-1195: deprecate and remove org.apache.kafka.streams.errors.BrokerNotFoundException

2025-07-23 Thread Chia-Ping Tsai
+1 (binding) Rajani Karuturi 於 2025年7月23日 週三 下午9:13寫道: > Hi All, > I would like to start a vote on KIP-1195: deprecate and remove > org.apache.kafka.streams.errors.BrokerNotFoundException > > KIP Link: https://cwiki.apache.org/confluence/x/8AxJFg > Discussion thread: > https://lists.apache.org/t

Re: [DISCUSS] KIP-1195: cleanup deprecated exceptions in org.apache.kafka.streams.errors.*

2025-07-23 Thread Chia-Ping Tsai
; Thanks, > ~ Rajani > > > On Tue, Jul 22, 2025 at 3:22 PM Chia-Ping Tsai wrote: > > > hi Rajani > > > > could you please add the "remove directly" to the Rejected Alternatives > > section? > > > > Best, > > Chia-Ping &

[jira] [Created] (KAFKA-19535) add integration tests for DescribeProducersOptions#brokerId

2025-07-22 Thread Chia-Ping Tsai (Jira)
Chia-Ping Tsai created KAFKA-19535: -- Summary: add integration tests for DescribeProducersOptions#brokerId Key: KAFKA-19535 URL: https://issues.apache.org/jira/browse/KAFKA-19535 Project: Kafka

Re: [DISCUSS] KIP-1160: Enable returning supported features from a specific broker

2025-07-22 Thread Chia-Ping Tsai
hi PoAn thanks for this KIP. Please take a look at following questions. chia_00: The behavior of "-1" defined by `DescribeFeaturesOptions#nodeId` is too detailed. That could obstructs us from tweaking the behavior in the future. Perhaps we could use "arbitrary controller/broker" instead. chia

[jira] [Resolved] (KAFKA-19213) Kafka java client ignores default properties

2025-07-22 Thread Chia-Ping Tsai (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-19213?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Chia-Ping Tsai resolved KAFKA-19213. Fix Version/s: 4.2.0 Resolution: Fixed > Kafka java client ignores defa

Re: [VOTE] 4.1.0 RC1

2025-07-22 Thread Chia-Ping Tsai
hi If anyone wants to test 4.1.0-RC1 without the backport fix, please simply configure the jdk_version to use another image. For example: jdk_version="openjdk:17-bullseye" /bin/bash tests/docker/run_tests.sh Best, Chia-Ping Josep Prat 於 2025年7月22日 週二 下午10:56寫道: > Hi Mickael, > I don't think we

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

2025-07-22 Thread Chia-Ping Tsai
empty and > remote.storage.enable is true, the local log segments will be cleaned based > on log.local.retention.bytes and log.local.retention.ms. > > Jun > > On Fri, Jul 18, 2025 at 9:52 AM Chia-Ping Tsai wrote: > > > > > > > JR34: For these two configurat

Re: [DISCUSS] KIP-1195: cleanup deprecated exceptions in org.apache.kafka.streams.errors.*

2025-07-22 Thread Chia-Ping Tsai
//cwiki.apache.org/confluence/display/KAFKA/KIP-1195%3A+deprecate+and+remove+org.apache.kafka.streams.errors.BrokerNotFoundException > > > > > > Thanks, > > ~ Rajani > > > > > > On Sun, Jul 20, 2025 at 8:36 PM Chia-Ping Tsai > > wrote: > > > >> hi Rajani > >> >

Re: [VOTE] KIP-1192: Add include argument to ConsumerPerformance tool

2025-07-22 Thread Chia-Ping Tsai
+1 (binding) Omnia Ibrahim 於 2025年7月22日 週二 下午4:52寫道: > Thanks for the KIP Federico. > > +1 binding > > Omnia > > > On 21 Jul 2025, at 16:22, Kirk True wrote: > > > > Thanks Fede! +1 (non-binding) > > > > On Sat, Jul 19, 2025, at 6:44 AM, Andrew Schofield wrote: > >> Hi, > >> Thanks for the KIP.

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

2025-07-22 Thread Chia-Ping Tsai
hi Andrew Thanks for this KIP. some questions are listed below. chia_00: Should we introduce `--consumer-property` to `kafka-consumer-perf-test.sh` and `kafka-share-consumer-perf-test.sh` since another perf tool, `kafka-producer-perf-test.sh`, has `--producer-property`? chia_01: should we al

[jira] [Created] (KAFKA-19533) correct the docs of "messages" for `kafka-consumer-perf-test.sh` and `kafka-share-consumer-perf-test.sh`

2025-07-22 Thread Chia-Ping Tsai (Jira)
Chia-Ping Tsai created KAFKA-19533: -- Summary: correct the docs of "messages" for `kafka-consumer-perf-test.sh` and `kafka-share-consumer-perf-test.sh` Key: KAFKA-19533 URL: https://issues.apache.org/j

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

2025-07-22 Thread Chia-Ping Tsai
hi Kuan-Po The method `updateClusterMetadata` hasn't been implemented for over three years after kraft was introduced. It seems we still function well without it. If the method is rarely used, introducing so many new public APIs for it seems a bit overkill. We could file a KIP to just deprecate

[jira] [Resolved] (KAFKA-13869) Update quota callback metadata in KRaft

2025-07-22 Thread Chia-Ping Tsai (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-13869?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Chia-Ping Tsai resolved KAFKA-13869. Resolution: Duplicate > Update quota callback metadata in KR

[jira] [Created] (KAFKA-19527) improve the docs of LogDirDescription for remote stroage

2025-07-20 Thread Chia-Ping Tsai (Jira)
Chia-Ping Tsai created KAFKA-19527: -- Summary: improve the docs of LogDirDescription for remote stroage Key: KAFKA-19527 URL: https://issues.apache.org/jira/browse/KAFKA-19527 Project: Kafka

Re: [DISCUSS] KIP-1187: Support to retrieve remote log size via DescribeLogDirs RPC

2025-07-20 Thread Chia-Ping Tsai
hi PoAn chia_00: should `setIncludeRemoteInfo(boolean)` be renamed to ` includeRemoteInfo(boolean)`? chia_01: the `PartitionSize` does not include the `RemoteLogSize`, right? chia_02: Should we add similar fields for `TotalBytes` and `UsableBytes`? Best, Chia-Ping ally heev 於 2025年7月17日 週四

[jira] [Created] (KAFKA-19526) Enable to use DescribeTopicPartitionsRequest with topic id

2025-07-20 Thread Chia-Ping Tsai (Jira)
Chia-Ping Tsai created KAFKA-19526: -- Summary: Enable to use DescribeTopicPartitionsRequest with topic id Key: KAFKA-19526 URL: https://issues.apache.org/jira/browse/KAFKA-19526 Project: Kafka

Re: [DISCUSS] KIP-1172: Improve EndToEndLatency tool

2025-07-20 Thread Chia-Ping Tsai
estion: > > When representing the value size in the header, do you think it’s better to > use the name ‘record-header-value-size’ or keep it as ‘record-header-size’? > > > Appreciate your thoughts. > > > Regards, > > Nick > > On Thu, Jul 17, 2025 at 9:03 PM C

Re: [DISCUSS] KIP-1195: cleanup deprecated exceptions in org.apache.kafka.streams.errors.*

2025-07-20 Thread Chia-Ping Tsai
Exception as > > deprecated and raise a PR which can be merged to master for the next 4.x > > release. > > How do we handle other deprecated methods from the exception handlers? > > Should I raise a PR and keep it unmerged until the 5.x release cycle > > starts? > &

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

2025-07-18 Thread Chia-Ping Tsai
> > JR34: For these two configurations, setting an empty list feels a bit > unintuitive. If an empty list is > provided, the consumer will call the unsubscribe method, which doesn't > seem appropriate given > the documentation states: "Topic-partitions assigned to this task to > replicate." > agre

Re: [VOTE] KIP-1100: Rename org.apache.kafka.server:type=AssignmentsManager and org.apache.kafka.storage.internals.log.RemoteStorageThreadPool metrics

2025-07-18 Thread Chia-Ping Tsai
+1(binding) > 黃竣陽 於 2025年7月18日 晚上7:18 寫道: > > Hello everyone, > > I would like to start a vote on KIP-1100 Rename > org.apache.kafka.server:type=AssignmentsManager > and org.apache.kafka.storage.internals.log.RemoteStorageThreadPool metrics > > KIP Link: https://cwiki.apache.org/confluence/x

Re: [VOTE] KIP-1193: Deprecate MX4j support

2025-07-18 Thread Chia-Ping Tsai
+1 (binding) > Federico Valeri 於 2025年7月18日 下午6:12 寫道: > > Hello, I would like to call a vote for KIP-1193: > > https://cwiki.apache.org/confluence/x/dAxJFg > > Thanks > Fede

Re: [DISCUSS] KIP-1100: Consider renaming org.apache.kafka.server:type=AssignmentsManager

2025-07-17 Thread Chia-Ping Tsai
hi Jiunn-Yang I think this KIP is ready for a VOTE thread. thanks, Chia-Ping 黃竣陽 於 2025年5月30日 週五 上午11:17寫道: > Hi all, > > I’d like to manually bump this thread. If there’s no further feedback, > I’ll start the vote tomorrow. > > Best Regards, > Jiunn-Yang > > > 黃竣陽 於 2025年5月11日 下午4:29 寫道: > >

[jira] [Created] (KAFKA-19518) Remove the usage of KafkaMetricsGroup(Class klass)

2025-07-17 Thread Chia-Ping Tsai (Jira)
Chia-Ping Tsai created KAFKA-19518: -- Summary: Remove the usage of KafkaMetricsGroup(Class klass) Key: KAFKA-19518 URL: https://issues.apache.org/jira/browse/KAFKA-19518 Project: Kafka Issue

[jira] [Created] (KAFKA-19517) LoadSummary#numBytes include the size of all records, but LoadSummary#numRecords only includes non-controller records

2025-07-17 Thread Chia-Ping Tsai (Jira)
Chia-Ping Tsai created KAFKA-19517: -- Summary: LoadSummary#numBytes include the size of all records, but LoadSummary#numRecords only includes non-controller records Key: KAFKA-19517 URL: https://issues.apache.org

Re: [DISCUSS] KIP-1195: cleanup deprecated exceptions in org.apache.kafka.streams.errors.*

2025-07-17 Thread Chia-Ping Tsai
> there is also no damage if we only deprecate it for now, and wait for 5.0 release to remove it. agreed. Keeping it in 4.x won't burn out the kafka server, so let's follow the deprecation cycle. Matthias J. Sax 於 2025年7月18日 週五 上午12:17寫道: > Thanks for the KIP Rajani. > > `BrokerNotFoundExceptio

Re: [DISCUSS] KIP-1172: Improve EndToEndLatency tool

2025-07-17 Thread Chia-Ping Tsai
), > > StandardCharsets.UTF_8); > > > > Header sentHeader = headers.iterator().next(); > > Header readHeader = records.iterator().next().headers().iterator().next(); > > > > if (!readHeader.equals(sentHeader)) { > > throw new RuntimeException(); > > > > i

[jira] [Resolved] (KAFKA-19501) System tests should use 17-bullseye instead of 17-buster

2025-07-16 Thread Chia-Ping Tsai (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-19501?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Chia-Ping Tsai resolved KAFKA-19501. Fix Version/s: 4.2.0 Resolution: Fixed > System tests should use 17-bulls

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

2025-07-16 Thread Chia-Ping Tsai
er.worker.verification.algorithms > > > > > > RestServerConfig > > > rest.extension.classes > > > admin.listeners > > > listeners > > > > > > Thanks, > > > > > > Jun > > > > > > On Tue, Jul 15, 2025 at 6

[jira] [Resolved] (KAFKA-19427) The __consumer_offsets topic applies the broker configuration message.max.bytes, which may cause the coordinator broker to allocate too much memory and cause OOM

2025-07-16 Thread Chia-Ping Tsai (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-19427?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Chia-Ping Tsai resolved KAFKA-19427. Resolution: Fixed > The __consumer_offsets topic applies the broker configurat

Re: [VOTE] 4.1.0 RC0

2025-07-16 Thread Chia-Ping Tsai
hi Mickael I'm going to merge the patch for https://issues.apache.org/jira/browse/KAFKA-19427. The bug could lead to an OOM in 4.x. The risk is low since the patch is small. Hence, I plan to backport it to 4.1.0 if you agree to accept it. Best, Chia-Ping On 2025/07/15 08:43:40 Mickael Maison

[jira] [Resolved] (KAFKA-19305) Make ClientQuotaImage and TopicImage immutable

2025-07-15 Thread Chia-Ping Tsai (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-19305?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Chia-Ping Tsai resolved KAFKA-19305. Fix Version/s: 4.2.0 Resolution: Fixed > Make ClientQuotaImage and TopicIm

Re: [DISCUSS] KIP-1193: Deprecate MX4j support

2025-07-15 Thread Chia-Ping Tsai
hi Fede Thanks for this KIP I think JMX Prometheus Exporter is the most popular choice these days, as it integrates well with modern monitoring stacks (Prometheus + Grafana). A similar alternative could be Jolokia + Hawtio, but I haven't seen this stack used much with Kafka. In both cases, a Jav

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

2025-07-14 Thread Chia-Ping Tsai
;>> support the case of "no supported cipher suite," as it doesn't make > > >> sense. > > >>>> > > >>>> In short, changing the default value from null to empty does not > break > > >>> the behavior. > > &

Re: [ANNOUNCE] New Kafka PMC member: Lucas Brutschy

2025-07-14 Thread Chia-Ping Tsai
Congratulations on your well-deserved new title! > Kamal Chandraprakash 於 2025年7月15日 上午11:39 寫道: > > Congrats Lucas! > >> On Tue, Jul 15, 2025, 08:24 黃竣陽 wrote: >> >> Congratulations Lucas! >> >> Best, >> Jiunn-Yang >> PoAn Yang 於 2025年7月15日 上午9:53 寫道: >>> >>> Congratulations Lucas

[jira] [Resolved] (KAFKA-19254) Add generic feature level metric

2025-07-14 Thread Chia-Ping Tsai (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-19254?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Chia-Ping Tsai resolved KAFKA-19254. Resolution: Fixed > Add generic feature level met

[jira] [Resolved] (KAFKA-19435) Optimize `kafka-consumer-groups.sh` to return the offset info of other partitions even when the leader of some partitions are missing

2025-07-14 Thread Chia-Ping Tsai (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-19435?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Chia-Ping Tsai resolved KAFKA-19435. Fix Version/s: 4.2.0 Resolution: Fixed > Optimize `kafka-consumer-groups.sh`

Re: MX4j support

2025-07-14 Thread Chia-Ping Tsai
+1 to deprecate mx4j and the project is EOL > Federico Valeri 於 2025年7月14日 晚上8:09 寫道: > > Hello devs, Kafka has support for MX4j, that can be used to expose JMX > beans over HTTP. > > https://github.com/apache/kafka/blob/trunk/core/src/main/scala/kafka/server/KafkaRaftServer.scala#L91 > > Thi

[jira] [Created] (KAFKA-19501) System tests should use 17-bullseye instead of 17-buster

2025-07-14 Thread Chia-Ping Tsai (Jira)
Chia-Ping Tsai created KAFKA-19501: -- Summary: System tests should use 17-bullseye instead of 17-buster Key: KAFKA-19501 URL: https://issues.apache.org/jira/browse/KAFKA-19501 Project: Kafka

[jira] [Created] (KAFKA-19500) kafka-consumer-groups.sh should fail quickly if the partition leader is unavailable

2025-07-14 Thread Chia-Ping Tsai (Jira)
Chia-Ping Tsai created KAFKA-19500: -- Summary: kafka-consumer-groups.sh should fail quickly if the partition leader is unavailable Key: KAFKA-19500 URL: https://issues.apache.org/jira/browse/KAFKA-19500

[ANNOUNCE] New Kafka Committer: PoAn Yang

2025-07-12 Thread Chia-Ping Tsai
Dear all, The Project Management Committee (PMC) for Apache Kafka is pleased to announce PoAn as a new Kafka committer. PoAn is a highly diligent and active contributor who consistently strives to improve our codebase. He has successfully completed numerous refactoring tasks and actively contribu

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

2025-07-10 Thread Chia-Ping Tsai
the type or default value should be fine in > > minor > > >>> release if we don't break the existing property files. > > I don't think this requires any compatibility considerations, so I’ve > > removed the Kafka 5.0 changes section. > > > > >&

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

2025-07-09 Thread Chia-Ping Tsai (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-19466?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Chia-Ping Tsai resolved KAFKA-19466. Fix Version/s: 4.2.0 Resolution: Fixed > LogConcurrencyTest should close the

[jira] [Created] (KAFKA-19488) Update the docs of "if-not-exists"

2025-07-09 Thread Chia-Ping Tsai (Jira)
Chia-Ping Tsai created KAFKA-19488: -- Summary: Update the docs of "if-not-exists" Key: KAFKA-19488 URL: https://issues.apache.org/jira/browse/KAFKA-19488 Project: Kafka

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

2025-07-09 Thread Chia-Ping Tsai
a compatibility column in the table. If a > value is formally disallowed, it would be useful to compare the old and the > new behavior (e.g. a different exception is thrown, etc). > > Thanks, > > Jun > > On Tue, Jul 8, 2025 at 8:36 AM Chia-Ping Tsai wrote: >

[jira] [Created] (KAFKA-19486) Always use the latest version of kafka-topics.sh to create topics in system tests

2025-07-09 Thread Chia-Ping Tsai (Jira)
Chia-Ping Tsai created KAFKA-19486: -- Summary: Always use the latest version of kafka-topics.sh to create topics in system tests Key: KAFKA-19486 URL: https://issues.apache.org/jira/browse/KAFKA-19486

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

2025-07-08 Thread Chia-Ping Tsai
quot; type, but they are handled as a LIST. For > > example, `early.start.listeners`, `security.providers`. Could we change > > their type to LIST to benefit from this KIP? I mean to make them > > non-nullable. > I have also updated the KIP to include the tables detailing the chang

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

2025-07-07 Thread Chia-Ping Tsai
ed that an > > >> empty > > >>>>>> cleanup.policy implies infinite retention. In fact, if one leaves > > >>>>>> cleanup.policy empty, it actually breaks remote storage since it > > will > > >> stop > > >>>>>> the cle

[jira] [Resolved] (KAFKA-18977) Fix flaky streams_smoke_test

2025-07-07 Thread Chia-Ping Tsai (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-18977?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Chia-Ping Tsai resolved KAFKA-18977. Resolution: Duplicate see KAFKA-19429 > Fix flaky streams_smoke_t

Re: [DISCUSS] Apache Kafka 4.1.0 release

2025-07-07 Thread Chia-Ping Tsai
for another, I backported https://github.com/apache/kafka/commit/f14e60fc8fa9bbffcd1b46b899a0cbb17f24745d, which is just a test migration, to reduce the conflicts when backporting KAFKA-19397 On 2025/07/07 08:22:54 Chia-Ping Tsai wrote: > hi Mickael > > I have merged https://github.c

Re: [DISCUSS] Apache Kafka 4.1.0 release

2025-07-07 Thread Chia-Ping Tsai
hi Mickael I have merged https://github.com/apache/kafka/pull/20112 to enable CI to run for branch 4.1 Also, I will backport KAFKA-19397 (https://github.com/apache/kafka/pull/20109) to branch 4.1 if CI looks good. Best, Chia-Ping On 2025/06/10 12:53:16 Mickael Maison wrote: > Hi, > > We are

[jira] [Created] (KAFKA-19475) updateQuotaMetricConfigs could iterate through all metrics under a write lock

2025-07-06 Thread Chia-Ping Tsai (Jira)
Chia-Ping Tsai created KAFKA-19475: -- Summary: updateQuotaMetricConfigs could iterate through all metrics under a write lock Key: KAFKA-19475 URL: https://issues.apache.org/jira/browse/KAFKA-19475

[jira] [Resolved] (KAFKA-19315) Move ControllerMutationQuotaManager to server module

2025-07-06 Thread Chia-Ping Tsai (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-19315?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Chia-Ping Tsai resolved KAFKA-19315. Fix Version/s: 4.2.0 Resolution: Fixed > Move ControllerMutationQuotaManager

Re: [FYI] Tiny wiki KIP list cleanup

2025-07-02 Thread Chia-Ping Tsai
Thanks for the cleanup! Kirk True 於 2025年7月2日 週三 上午2:56寫道: > Hi all, > > I moved a dozen accepted KIPs from the "KIPs under discussion" to the > "Adopted KIPs" table. > > Here are the diffs if you're interested/concerned: > > > https://cwiki.apache.org/confluence/pages/diffpagesbyversion.action?

[jira] [Reopened] (KAFKA-19221) IOException on log segment close shouldn't be ignored

2025-06-30 Thread Chia-Ping Tsai (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-19221?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Chia-Ping Tsai reopened KAFKA-19221: > IOException on log segment close shouldn't be

[jira] [Resolved] (KAFKA-19402) Typo in EventAccumulator.java

2025-06-29 Thread Chia-Ping Tsai (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-19402?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Chia-Ping Tsai resolved KAFKA-19402. Fix Version/s: 4.2.0 Resolution: Fixed > Typo in EventAccumulator.j

[jira] [Resolved] (KAFKA-18486) Remove ReplicaManager#becomeLeaderOrFollower

2025-06-29 Thread Chia-Ping Tsai (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-18486?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Chia-Ping Tsai resolved KAFKA-18486. Fix Version/s: 4.2.0 Resolution: Fixed There are some follow-up cleanups, but the

[jira] [Created] (KAFKA-19447) makeFollower and makeLeader should use PartitionRegistration instead of PartitionState

2025-06-29 Thread Chia-Ping Tsai (Jira)
Chia-Ping Tsai created KAFKA-19447: -- Summary: makeFollower and makeLeader should use PartitionRegistration instead of PartitionState Key: KAFKA-19447 URL: https://issues.apache.org/jira/browse/KAFKA-19447

[jira] [Resolved] (KAFKA-19420) Don't export SocketServer from ClusterInstance

2025-06-27 Thread Chia-Ping Tsai (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-19420?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Chia-Ping Tsai resolved KAFKA-19420. Fix Version/s: 4.2.0 Resolution: Fixed > Don't export SocketSer

Re: 4.0.1 bug-fix release?

2025-06-27 Thread Chia-Ping Tsai
Thanks Christo Luke Chen 於 2025年6月27日 週五 下午7:13寫道: > +1 > Thanks Christo! > > Let me know if you encounter any problem. > > Luke > > On Fri, Jun 27, 2025 at 6:12 PM Mickael Maison > wrote: > > > +1 > > > > Thanks Christo > > > > On Thu, Jun 26, 2025 at 10:58 AM Bruno Cadonna > wrote: > > >

[jira] [Resolved] (KAFKA-18834) Fix LoggingResourceTest#testSetLevelDefaultScope

2025-06-25 Thread Chia-Ping Tsai (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-18834?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Chia-Ping Tsai resolved KAFKA-18834. Fix Version/s: 4.2.0 Resolution: Fixed > Fix LoggingResourceT

[jira] [Resolved] (KAFKA-18656) Backport KAFKA-18597 to 4.0

2025-06-25 Thread Chia-Ping Tsai (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-18656?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Chia-Ping Tsai resolved KAFKA-18656. Fix Version/s: 4.0.1 Resolution: Fixed > Backport KAFKA-18597 to

Re: [ANNOUNCE] New Kafka Committer: Omnia Ibrahim

2025-06-25 Thread Chia-Ping Tsai
Thanks to Omnia for all your contributions! Apoorv Mittal 於 2025年6月25日 週三 下午11:43寫道: > Congrats Omnia! Great to hear! > > Regards, > Apoorv Mittal > > > On Wed, Jun 25, 2025 at 4:39 PM Kuan-Po Tseng wrote: > > > Congrats Omnia! > > Well deserved. > > > > On Wed, Jun 25, 2025 at 11:34 PM Josep P

[jira] [Resolved] (KAFKA-18778) Fix the inconsistent lastest supported version in StorageTool.scala and FutureCommand

2025-06-24 Thread Chia-Ping Tsai (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-18778?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Chia-Ping Tsai resolved KAFKA-18778. Fix Version/s: 4.2.0 Resolution: Fixed > Fix the inconsistent lastest suppor

[jira] [Resolved] (KAFKA-19407) OffsetsOutOfOrderException on followers due to the race condition in the leader

2025-06-24 Thread Chia-Ping Tsai (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-19407?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Chia-Ping Tsai resolved KAFKA-19407. Resolution: Fixed trunk: https://github.com/apache/kafka/commit

[jira] [Created] (KAFKA-19428) IOException during writing max timestamp should not be ignored

2025-06-23 Thread Chia-Ping Tsai (Jira)
Chia-Ping Tsai created KAFKA-19428: -- Summary: IOException during writing max timestamp should not be ignored Key: KAFKA-19428 URL: https://issues.apache.org/jira/browse/KAFKA-19428 Project: Kafka

[jira] [Resolved] (KAFKA-19324) org.apache.kafka.common.test.TestUtils should be package-private

2025-06-22 Thread Chia-Ping Tsai (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-19324?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Chia-Ping Tsai resolved KAFKA-19324. Fix Version/s: 4.2.0 Resolution: Fixed > org.apache.kafka.common.test.TestUt

[jira] [Resolved] (KAFKA-18926) `KafkaPrincipalBuilder` should extend `KafkaPrincipalSerde`

2025-06-22 Thread Chia-Ping Tsai (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-18926?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Chia-Ping Tsai resolved KAFKA-18926. Fix Version/s: 4.2.0 Resolution: Fixed > `KafkaPrincipalBuilder` should ext

[jira] [Resolved] (KAFKA-19140) ConnectAssignor#performAssignment second parameter can be replace from String to ConnectProtocolCompatibility

2025-06-19 Thread Chia-Ping Tsai (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-19140?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Chia-Ping Tsai resolved KAFKA-19140. Fix Version/s: 4.2.0 Resolution: Fixed > ConnectAssignor#performAssignment sec

[jira] [Resolved] (KAFKA-18903) Use curl + jq in update-cache.sh

2025-06-19 Thread Chia-Ping Tsai (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-18903?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Chia-Ping Tsai resolved KAFKA-18903. Fix Version/s: 4.2.0 Resolution: Fixed > Use curl + jq in update-cache

[jira] [Resolved] (KAFKA-19239) Rewrite IntegrationTestUtils by java

2025-06-19 Thread Chia-Ping Tsai (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-19239?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Chia-Ping Tsai resolved KAFKA-19239. Fix Version/s: 4.2.0 Resolution: Fixed > Rewrite IntegrationTestUtils by j

[jira] [Resolved] (KAFKA-19406) Remove BrokerTopicStats#removeOldFollowerMetrics

2025-06-19 Thread Chia-Ping Tsai (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-19406?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Chia-Ping Tsai resolved KAFKA-19406. Fix Version/s: 4.2.0 Resolution: Fixed > Remove BrokerTopicSt

[jira] [Created] (KAFKA-19420) Don't export SocketServer from ClusterInstance

2025-06-19 Thread Chia-Ping Tsai (Jira)
Chia-Ping Tsai created KAFKA-19420: -- Summary: Don't export SocketServer from ClusterInstance Key: KAFKA-19420 URL: https://issues.apache.org/jira/browse/KAFKA-19420 Project: Kafka

[jira] [Resolved] (KAFKA-18627) add allowed modules to JaasUtils

2025-06-18 Thread Chia-Ping Tsai (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-18627?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Chia-Ping Tsai resolved KAFKA-18627. Resolution: Fixed > add allowed modules to JaasUt

[jira] [Resolved] (KAFKA-19350) don't propagate the error caused by CreateTopicPolicy to FatalFaultHandler

2025-06-17 Thread Chia-Ping Tsai (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-19350?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Chia-Ping Tsai resolved KAFKA-19350. Fix Version/s: 4.2.0 Resolution: Fixed > don't propagate the error c

[jira] [Resolved] (KAFKA-14895) Move AddPartitionsToTxnManager files to java

2025-06-17 Thread Chia-Ping Tsai (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-14895?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Chia-Ping Tsai resolved KAFKA-14895. Fix Version/s: 4.2.0 Resolution: Fixed > Move AddPartitionsToTxnManager files

Re: [ANNOUNCE] New Kafka PMC Member: José Armando García Sancio

2025-06-16 Thread Chia-Ping Tsai
Congrats José!!!

Re: [VOTE] KIP-1153: Refactor Kafka Streams CloseOptions to Fluent API Style

2025-06-13 Thread Chia-Ping Tsai
+1 (binding) On 2025/05/20 12:38:06 黃竣陽 wrote: > Hello all, > > I would like to call a vote on KIP-1153: Refactor Kafka Streams CloseOptions > to Fluent API Style > > KIP: > > > Discussion thread: >

[jira] [Resolved] (KAFKA-17607) Add CI step to verify LICENSE-binary

2025-06-13 Thread Chia-Ping Tsai (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-17607?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Chia-Ping Tsai resolved KAFKA-17607. Fix Version/s: 4.1.0 Resolution: Fixed > Add CI step to verify LICENSE-bin

[jira] [Created] (KAFKA-19406) Remove BrokerTopicStats#removeOldFollowerMetrics

2025-06-12 Thread Chia-Ping Tsai (Jira)
Chia-Ping Tsai created KAFKA-19406: -- Summary: Remove BrokerTopicStats#removeOldFollowerMetrics Key: KAFKA-19406 URL: https://issues.apache.org/jira/browse/KAFKA-19406 Project: Kafka Issue

Re: Need help with kafka contribution

2025-06-12 Thread Chia-Ping Tsai
hi Palak I will take a look later. Pleas be patient :) Best, Chia-Ping Palak Kapoor 於 2025年6月12日 週四 下午2:46寫道: > Hi, > I am a newbie contributing to kafka. Worked on below basic JIRA issues. > Whom should I ask for a review? > > https://issues.apache.org/jira/browse/KAFKA-19399 > https://issues

[jira] [Created] (KAFKA-19401) Consider keeping/removing the internal wrappers from streams module

2025-06-11 Thread Chia-Ping Tsai (Jira)
Chia-Ping Tsai created KAFKA-19401: -- Summary: Consider keeping/removing the internal wrappers from streams module Key: KAFKA-19401 URL: https://issues.apache.org/jira/browse/KAFKA-19401 Project

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

2025-06-11 Thread Chia-Ping Tsai
s, > > Jiunn-Yang > > > > > 黃竣陽 於 2025年5月8日 晚上8:35 寫道: > > > > > > Hi all, > > > > > > Thanks for all the feedback. I’ve updated the KIP and introduced a > > > CloseOptionsInternal class that provides a getter method. > > > >

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

2025-06-11 Thread Chia-Ping Tsai
dback, > > I’ll start the vote tomorrow. > > > > Best Regards, > > Jiunn-Yang > > > > > 黃竣陽 於 2025年5月8日 晚上8:35 寫道: > > > > > > Hi all, > > > > > > Thanks for all the feedback. I’ve updated the KIP and introduced a > > >

[jira] [Resolved] (KAFKA-19391) Update the opentelemetry-proto and protobuf

2025-06-10 Thread Chia-Ping Tsai (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-19391?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Chia-Ping Tsai resolved KAFKA-19391. Fix Version/s: 4.2.0 Resolution: Fixed > Update the opentelemetry-proto

Re: [DISCUSS] Apache Kafka 4.1.0 release

2025-06-10 Thread Chia-Ping Tsai
hi Mickael Thanks for managing the 4.1 branch. Are there any rules of backporting to 4.1 that we need to know? Best, Chia-Ping Mickael Maison 於 2025年6月10日 週二 下午11:47寫道: > Hi, > > With the help of Infra I was able to create the new 4.1 branch: > https://github.com/apache/kafka/tree/4.1 > I'll

Re: [DISCUSS] KIP-1172: Improve EndToEndLatency tool

2025-06-10 Thread Chia-Ping Tsai
hi Nick thanks for this proposal. Some questions are listed below. chia_0: This tool is used by e2e, so could you please describe the changes for e2e too? chia_1: the two new fields (*message-key-size-byt and * *message-header-size-bytes)* are NOT supported by old (index) arguments, right? If so

[jira] [Resolved] (KAFKA-19392) `metadata.log.segment.ms` is not applied

2025-06-10 Thread Chia-Ping Tsai (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-19392?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Chia-Ping Tsai resolved KAFKA-19392. Resolution: Fixed > `metadata.log.segment.ms` is not appl

[jira] [Created] (KAFKA-19392) `metadata.log.segment.ms` is not applied

2025-06-09 Thread Chia-Ping Tsai (Jira)
Chia-Ping Tsai created KAFKA-19392: -- Summary: `metadata.log.segment.ms` is not applied Key: KAFKA-19392 URL: https://issues.apache.org/jira/browse/KAFKA-19392 Project: Kafka Issue Type: Bug

[jira] [Created] (KAFKA-19391) Update the opentelemetry-proto and protobuf

2025-06-09 Thread Chia-Ping Tsai (Jira)
Chia-Ping Tsai created KAFKA-19391: -- Summary: Update the opentelemetry-proto and protobuf Key: KAFKA-19391 URL: https://issues.apache.org/jira/browse/KAFKA-19391 Project: Kafka Issue Type

[jira] [Created] (KAFKA-19388) Move EdgeCaseRequestTest to clients-integration-tests module

2025-06-07 Thread Chia-Ping Tsai (Jira)
Chia-Ping Tsai created KAFKA-19388: -- Summary: Move EdgeCaseRequestTest to clients-integration-tests module Key: KAFKA-19388 URL: https://issues.apache.org/jira/browse/KAFKA-19388 Project: Kafka

[jira] [Created] (KAFKA-19384) The passing of BrokerRegistrationRequestTest is a false positive

2025-06-06 Thread Chia-Ping Tsai (Jira)
Chia-Ping Tsai created KAFKA-19384: -- Summary: The passing of BrokerRegistrationRequestTest is a false positive Key: KAFKA-19384 URL: https://issues.apache.org/jira/browse/KAFKA-19384 Project: Kafka

[jira] [Created] (KAFKA-19382) Upgrade junit from 5.10 to 5.13

2025-06-06 Thread Chia-Ping Tsai (Jira)
Chia-Ping Tsai created KAFKA-19382: -- Summary: Upgrade junit from 5.10 to 5.13 Key: KAFKA-19382 URL: https://issues.apache.org/jira/browse/KAFKA-19382 Project: Kafka Issue Type: Improvement

  1   2   3   4   5   6   7   8   9   10   >