Re: [VOTE] KIP-1127: Flexible Windows for Late Arriving Data

2025-02-04 Thread Sophie Blee-Goldman
+1 (binding) Thanks for the KIP! Neat and practical idea On Tue, Feb 4, 2025 at 10:52 AM Almog Gavra wrote: > Hello All, > > I'd like to start a vote on KIP-1127. Please take a look at the KIP and > Discussion thread and let me know what you think. > > Note that th

[VOTE] KIP-1127: Flexible Windows for Late Arriving Data

2025-02-04 Thread Almog Gavra
Hello All, I'd like to start a vote on KIP-1127. Please take a look at the KIP and Discussion thread and let me know what you think. Note that the discussion thread was incorrectly prefixed with KIP-1124 instead of KIP-1127 (oops!). Link to KIP: https://cwiki.apache.org/confluence/display/

Re: [VOTE] KIP-896: Remove old client protocol API versions in Kafka 4.0

2025-01-28 Thread Ismael Juma
sumer > related api versions - fetch, group and offset management primarily (fetch > was the most impacted). > > Ismael > > On Tue, Nov 21, 2023 at 12:06 PM Ismael Juma wrote: > >> Hi all, >> >> I would like to start a vote on KIP-896. Please take a look and let us

Re: [VOTE] KIP-1125: Remove Invalid 'numberOfOpenFiles' Metric from RocksDB State Store

2025-01-21 Thread Swikar Patel
 Thanks everyone for voting and feedback. The vote has passed with 3 binding votes (Chia, Bruno and Matthias). Regards, Swikar > On Jan 21, 2025, at 9:26 PM, Chia-Ping Tsai wrote: > hi Swikar > > for example: https://lists.apache.org/thread/3qlw82pxkb6yb7ojk6f46wpglkhx4kzf

Re: [VOTE] KIP-1125: Remove Invalid 'numberOfOpenFiles' Metric from RocksDB State Store

2025-01-21 Thread Chia-Ping Tsai
hi Swikar for example: https://lists.apache.org/thread/3qlw82pxkb6yb7ojk6f46wpglkhx4kzf We need to count the binding and non-binding votes. Importantly, a KIP requires at least three binding votes to pass, so we usually announce the vote count to ensure its validity. Best, Chia-Ping

Re: [VOTE] KIP-1125: Remove Invalid 'numberOfOpenFiles' Metric from RocksDB State Store

2025-01-21 Thread Swikar Patel
How > On Jan 21, 2025, at 7:58 PM, Chia-Ping Tsai wrote: > > hi Swikar > > We typically announce the vote results during the final, so could you please > adhere to that pattern? > > Thanks, > Chia-Ping > > > >> Swikar Patel 於 2025年1月22日 凌晨4:11 寫

Re: [VOTE] KIP-1125: Remove Invalid 'numberOfOpenFiles' Metric from RocksDB State Store

2025-01-21 Thread Chia-Ping Tsai
hi Swikar We typically announce the vote results during the final, so could you please adhere to that pattern? Thanks, Chia-Ping > Swikar Patel 於 2025年1月22日 凌晨4:11 寫道: > > Okay Closed now and Updated KIP wiki page as [Accepted] > >> On Jan 21, 2025, at 11:01 AM, Matt

Re: [VOTE] KIP-1125: Remove Invalid 'numberOfOpenFiles' Metric from RocksDB State Store

2025-01-21 Thread Swikar Patel
Okay Closed now and Updated KIP wiki page as [Accepted] > On Jan 21, 2025, at 11:01 AM, Matthias J. Sax wrote: > > I think we can close the vote on this KIP? > >> On 1/13/25 2:52 PM, Swikar Patel wrote: >> Thanks Chia-Ping I have now added that to KIP. >>>&g

Re: [VOTE] KIP-1125: Remove Invalid 'numberOfOpenFiles' Metric from RocksDB State Store

2025-01-21 Thread Matthias J. Sax
I think we can close the vote on this KIP? On 1/13/25 2:52 PM, Swikar Patel wrote: Thanks Chia-Ping I have now added that to KIP. On Jan 13, 2025, at 2:43 PM, Chia-Ping Tsai wrote:  +1(binding) Could you please add the discussion of jira to KIP-1125 to remind readers that discussion

Re: [VOTE] KIP-1091: Improved Kafka Streams operator metrics

2025-01-17 Thread Matthias J. Sax
Nov 11, 2024 at 12:59 PM Bill Bejeck wrote: Hi All, The vote is now closed. KIP-1091 has been accepted with 3 binding votes (Lucas, Matthias, and Sophie) and 1 non-binding vote (Apoorv). Thanks to everyone for participating. -Bill On Wed, Nov 6, 2024 at 11:40 PM Sophie Blee-Goldman wrote:

[jira] [Resolved] (KAFKA-17643) Response handling for pre-vote set to True

2025-01-17 Thread Jira
[ https://issues.apache.org/jira/browse/KAFKA-17643?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] José Armando García Sancio resolved KAFKA-17643. Resolution: Fixed > Response handling for pre-vote set to T

Re: [VOTE] KIP-1125: Remove Invalid 'numberOfOpenFiles' Metric from RocksDB State Store

2025-01-14 Thread Bruno Cadonna
discussion phase has been completed in the jira ? Thanks, Chia-Ping Matthias J. Sax 於 2025年1月14日 清晨6:20 寫道: +1 (binding) On 1/13/25 12:17 PM, Swikar Patel wrote: Hi Kafka team, Hope you all good! I’ve proposed KIP and I'd like to start the vote on KIP-1125: Remove In

Re: [VOTE] KIP-1125: Remove Invalid 'numberOfOpenFiles' Metric from RocksDB State Store

2025-01-13 Thread Swikar Patel
; > Thanks, > Chia-Ping > >> Matthias J. Sax 於 2025年1月14日 清晨6:20 寫道: >> >> +1 (binding) >> >>>> On 1/13/25 12:17 PM, Swikar Patel wrote: >>> Hi Kafka team, >>> Hope you all good! >>> I’ve proposed KIP and I'd like to start the

Re: [VOTE] KIP-1125: Remove Invalid 'numberOfOpenFiles' Metric from RocksDB State Store

2025-01-13 Thread Chia-Ping Tsai
: >> Hi Kafka team, >> Hope you all good! >> I’ve proposed KIP and I'd like to start the vote on KIP-1125: Remove Invalid >> 'numberOfOpenFiles' Metric from RocksDB State Store >> <https://cwiki.apache.org/confluence/display/KAFKA/KIP-1125%3A+Remo

Re: [VOTE] KIP-1125: Remove Invalid 'numberOfOpenFiles' Metric from RocksDB State Store

2025-01-13 Thread Matthias J. Sax
+1 (binding) On 1/13/25 12:17 PM, Swikar Patel wrote: Hi Kafka team, Hope you all good! I’ve proposed KIP and I'd like to start the vote on KIP-1125: Remove Invalid 'numberOfOpenFiles' Metric from RocksDB State Store <https://cwiki.apache.org/confluence/display/KAFKA/

[VOTE] KIP-1125: Remove Invalid 'numberOfOpenFiles' Metric from RocksDB State Store

2025-01-13 Thread Swikar Patel
Hi Kafka team, Hope you all good! I’ve proposed KIP and I'd like to start the vote on KIP-1125: Remove Invalid 'numberOfOpenFiles' Metric from RocksDB State Store <https://cwiki.apache.org/confluence/display/KAFKA/KIP-1125%3A+Remove+Invalid+%27numberOfOpenFiles%27+Metric+

[jira] [Created] (KAFKA-18438) Pre-Vote followups

2025-01-07 Thread Alyssa Huang (Jira)
Alyssa Huang created KAFKA-18438: Summary: Pre-Vote followups Key: KAFKA-18438 URL: https://issues.apache.org/jira/browse/KAFKA-18438 Project: Kafka Issue Type: Task Reporter

Re: [VOTE] KIP-1034: Dead letter queue in Kafka Streams

2025-01-07 Thread Sebastien Viale
Thanks for voting. I'll conclude the vote with 3 binding votes from: Matthias, Bruno and Lucas. cheers ! Damien, Loic and Sébastien De : Lucas Brutschy Envoyé : vendredi 3 janvier 2025 15:23 À : dev@kafka.apache.org Objet : [EXT] Re: [VOTE] KIP-1034:

Re: [VOTE] KIP-1098: Reverse Checkpointing in MirrorMaker

2025-01-06 Thread Dániel Urbán
g the KIP and voting. >>> Daniel >>> >>> Dániel Urbán ezt írta (időpont: 2024. dec. 5., >>> Cs, 12:08): >>> >>>> Bumping this vote - the change has a relatively small footprint, but >>>> fills a sizable gap in MM2. >&g

Re: [VOTE] KIP-1034: Dead letter queue in Kafka Streams

2025-01-03 Thread Lucas Brutschy
+1 (binding) on the updated KIP. Thanks for the updates! On Fri, Dec 20, 2024 at 2:28 AM Matthias J. Sax wrote: > > I think I did miss the train to vote originally, but I am > > +1 (binding) > > on the updated KIP. > > > -Matthias > > On 12/18/24 12:4

Re: [VOTE] KIP-1107: Adding topic-level acks and compressions for producers

2025-01-02 Thread TaiJu Wu
anges to the producer, and > it's essential to ensure we have thoroughly considered all perspectives. > > Best, > Chia-Ping > > On 2025/01/03 03:03:03 TaiJu Wu wrote: > > Hello everyone, > > > > Happy New Year! > > There is no response about KIP-1107 fo

Re: [VOTE] KIP-1107: Adding topic-level acks and compressions for producers

2025-01-02 Thread Chia-Ping Tsai
t; Hello everyone, > > Happy New Year! > There is no response about KIP-1107 for a long time so I would like to vote. > > Discussion thread: > https://lists.apache.org/thread/v1sm6c59j5wppg1w6t17bvkgqlt3orp7 > > KIP-1107 > https://cwiki.apache.org/confluence/display/KA

[VOTE] KIP-1107: Adding topic-level acks and compressions for producers

2025-01-02 Thread TaiJu Wu
Hello everyone, Happy New Year! There is no response about KIP-1107 for a long time so I would like to vote. Discussion thread: https://lists.apache.org/thread/v1sm6c59j5wppg1w6t17bvkgqlt3orp7 KIP-1107 https://cwiki.apache.org/confluence/display/KAFKA/KIP-1107%3A++Adding+topic-level+acks+and

Re: [VOTE] KIP-1118: Add Deadlock Protection on Producer Network Thread

2025-01-02 Thread TengYao Chi
Hello everyone, Happy New Year 😊 I'm writing to inform you that the vote is now closed. The KIP-1118 has been accepted with 3 binding +1s from Chia-Ping, Andrew, and Kamal, as well as 3 non-binding +1s from Kirk, TaiJu, and 郭骏旺. Thank you all for your participation! Sincerely, TengYao

Re: [VOTE] KIP-1118: Add Deadlock Protection on Producer Network Thread

2025-01-02 Thread Kamal Chandraprakash
+1 (binding). Thanks for the KIP! On Mon, Dec 23, 2024, 08:28 TengYao Chi wrote: > Hi everyone, > > As the vote has been pending for a week, I would like to bump it manually. > Thank you for your attention. > > Sincerely, > TengYao > > Andrew Schofield 於 2024年12月1

Re: [VOTE] KIP-1118: Add Deadlock Protection on Producer Network Thread

2024-12-22 Thread TengYao Chi
Hi everyone, As the vote has been pending for a week, I would like to bump it manually. Thank you for your attention. Sincerely, TengYao Andrew Schofield 於 2024年12月16日 週一 下午10:25寫道: > +1 (binding) > > > From: TaiJu Wu > Sent: 16 December 2024

Re: [VOTE] KIP-1020 Move `window.size.ms` and `windowed.inner.serde.class` from `StreamsConfig` to TimeWindowedDe/Serializer class

2024-12-21 Thread PoAn Yang
. Thanks, PoAn On 2024/04/19 17:40:58 Lucia Cerchie wrote: > Thanks all for voting. > > I'm now closing the vote. The vote passes with > - 3 +1 bindings votes from Lucas Brutschy, Sophie Blee-Goldman, and > Matthias J Sax > - 0 +1 non-binding votes > - 0 -1 votes >

Re: [VOTE] KIP-1034: Dead letter queue in Kafka Streams

2024-12-19 Thread Matthias J. Sax
I think I did miss the train to vote originally, but I am +1 (binding) on the updated KIP. -Matthias On 12/18/24 12:47 AM, Bruno Cadonna wrote: Hi, We came to a conclusion in the discussion thread and Sébastien updated the KIP. IMO, we can recast our votes. +1 (binding) Thanks for the

Re: [VOTE] KIP-1034: Dead letter queue in Kafka Streams

2024-12-18 Thread Bruno Cadonna
De : Matthias J. Sax Envoyé : samedi 7 décembre 2024 00:39 À : dev@kafka.apache.org Objet : [EXT] Re: [VOTE] KIP-1034: Dead letter queue in Kafka Streams Warning External sender Do not click on any links or open any attachments unless you trust the sender and know the content is safe. Thanks

Re: [VOTE] KIP-1098: Reverse Checkpointing in MirrorMaker

2024-12-17 Thread Dániel Urbán
t: 2024. dec. 9., H, > 9:06): > >> Gentle bump - please consider checking the KIP and voting. >> Daniel >> >> Dániel Urbán ezt írta (időpont: 2024. dec. 5., >> Cs, 12:08): >> >>> Bumping this vote - the change has a relatively small footprint, but &g

Re: [VOTE] KIP-1109: Unifying Kafka Consumer Topic Metrics

2024-12-17 Thread Apoorv Mittal
Thanks everyone for voting and feedback. The vote has passed with 3 binding votes (Chia, Andrew and Jun). Regards, Apoorv Mittal On Mon, Dec 16, 2024 at 6:25 PM Jun Rao wrote: > Hi, Apoorv, > > Thanks for the KIP. +1 > > Jun > > On Thu, Dec 12, 2024 at 4:26 PM A

Re: [VOTE] KIP-1101: Trigger rebalance on rack topology changes

2024-12-16 Thread 郭骏旺
+1 (non-binding) On Mon, Dec 16, 2024 at 5:01 PM PoAn Yang wrote: > Hi All, > > I would like to start vote on KIP-1101: > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-1101%3A+Trigger+rebalance+on+rack+topology+changes > > Discussion thread: > https:/

Re: [VOTE] KIP-1109: Unifying Kafka Consumer Topic Metrics

2024-12-16 Thread Jun Rao
Hi, Apoorv, Thanks for the KIP. +1 Jun On Thu, Dec 12, 2024 at 4:26 PM Apoorv Mittal wrote: > Hi All, > I would like to start vote on KIP-1109: > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-1109%3A+Unifying+Kafka+Consumer+Topic+Metrics > > Discussi

Re: [VOTE] KIP-1118: Add Deadlock Protection on Producer Network Thread

2024-12-16 Thread Andrew Schofield
+1 (binding) From: TaiJu Wu Sent: 16 December 2024 09:41 To: dev@kafka.apache.org Subject: Re: [VOTE] KIP-1118: Add Deadlock Protection on Producer Network Thread +1(non-binding) On Mon, Dec 16, 2024 at 5:41 PM Chia-Ping Tsai wrote: > +1 (binding) > &g

RE: [VOTE] KIP-1118: Add Deadlock Protection on Producer Network Thread

2024-12-16 Thread Jhen-Yung Hsu
+1 (non-binding)

Re: [VOTE] KIP-1118: Add Deadlock Protection on Producer Network Thread

2024-12-16 Thread TaiJu Wu
t; > Based on our discussion > > > <https://lists.apache.org/thread/8npgn4d7wwlwvdy1h4xpbdlffksstddl> > > > regarding > > > KIP-1118 > > > < > > > > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-1118%3A+Add+Deadlock+Protection+on+Producer+Network+Thread > > > >, > > > I believe this KIP is now ready for a vote. > > > > > > Sincerely, > > > TengYao > > > > > >

Re: [VOTE] KIP-1118: Add Deadlock Protection on Producer Network Thread

2024-12-16 Thread Chia-Ping Tsai
t; > regarding > > KIP-1118 > > < > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-1118%3A+Add+Deadlock+Protection+on+Producer+Network+Thread > > >, > > I believe this KIP is now ready for a vote. > > > > Sincerely, > > TengYao > > >

Re: [VOTE] KIP-1101: Trigger rebalance on rack topology changes

2024-12-16 Thread PoAn Yang
Hi Chia-Ping, Thanks for the vote and reminder. I update the KIP for it. Regards, PoAn > On Dec 16, 2024, at 5:19 PM, TaiJu Wu wrote: > > +1 (non-binding) > > On Mon, Dec 16, 2024 at 5:01 PM PoAn Yang wrote: > >> Hi All, >> >> I would like to

Re: [VOTE] KIP-1101: Trigger rebalance on rack topology changes

2024-12-16 Thread TaiJu Wu
+1 (non-binding) On Mon, Dec 16, 2024 at 5:01 PM PoAn Yang wrote: > Hi All, > > I would like to start vote on KIP-1101: > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-1101%3A+Trigger+rebalance+on+rack+topology+changes > > Discussion thread: > https:/

Re: [VOTE] KIP-1101: Trigger rebalance on rack topology changes

2024-12-16 Thread Chia-Ping Tsai
ll, > > I would like to start vote on KIP-1101: > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-1101%3A+Trigger+rebalance+on+rack+topology+changes > > Discussion thread: > https://lists.apache.org/thread/l8ko353v3nn1blgymsty895x6c98oxlx > > Thanks, > PoAn

[VOTE] KIP-1101: Trigger rebalance on rack topology changes

2024-12-16 Thread PoAn Yang
Hi All, I would like to start vote on KIP-1101: https://cwiki.apache.org/confluence/display/KAFKA/KIP-1101%3A+Trigger+rebalance+on+rack+topology+changes Discussion thread: https://lists.apache.org/thread/l8ko353v3nn1blgymsty895x6c98oxlx Thanks, PoAn

Re: [VOTE] KIP-1111: Enforcing Explicit Naming for Kafka Streams Internal Topics

2024-12-15 Thread Sebastien Viale
Hi, Thanks for voting. I'll conclude the vote with 5 binding votes from: Matthias, Sophie, Bruno, Bill and Lucas. cheers ! Sébastien De : Matthias J. Sax Envoyé : jeudi 12 décembre 2024 22:48 À : dev@kafka.apache.org Objet : [EXT] Re: [VOTE] KIP

Re: [VOTE] KIP-1118: Add Deadlock Protection on Producer Network Thread

2024-12-15 Thread 郭骏旺
P-1118%3A+Add+Deadlock+Protection+on+Producer+Network+Thread > >, > I believe this KIP is now ready for a vote. > > Sincerely, > TengYao >

Re: [VOTE] KIP-1068: New metrics for the new KafkaConsumer

2024-12-14 Thread PoAn Yang
> On Thu, Jul 25, 2024 at 6:25 AM Lianet M. wrote: > > >>> > > >>>> Hi Brenden, > > >>>> > > >>>> +1 (non-binding) from me. > > >>>> > > >>>> Thanks for the KIP! > > >>>>

RE: [VOTE] KIP-1118: Add Deadlock Protection on Producer Network Thread

2024-12-13 Thread Jhen-Yung Hsu
+1 (non-binding)

Re: [VOTE] KIP-1118: Add Deadlock Protection on Producer Network Thread

2024-12-13 Thread Kirk True
+Add+Deadlock+Protection+on+Producer+Network+Thread>, > I believe this KIP is now ready for a vote. > > Sincerely, > TengYao >

Re: [VOTE] KIP-1109: Unifying Kafka Consumer Topic Metrics

2024-12-13 Thread Andrew Schofield
Hi Apoorv, Thanks for the KIP. +1 (binding) Andrew From: Chia-Ping Tsai Sent: 13 December 2024 01:30 To: dev@kafka.apache.org Subject: Re: [VOTE] KIP-1109: Unifying Kafka Consumer Topic Metrics +1 (binding) Apoorv Mittal 於 2024年12月13日 週五 上午8:26寫道

Re: [VOTE]: KIP-1071: Streams Rebalance Protocol

2024-12-13 Thread Lucas Brutschy
com> wrote: > > > Hi Bruno and Lucas, > > Thanks for proposing this excellent KIP. > > > > +1 (binding) > > > > Thanks, > > Andrew > > > > From: Alieh Saeedi > > Sent: 03 December 2024 16:07 >

Re: [VOTE] KIP-1098: Reverse Checkpointing in MirrorMaker

2024-12-13 Thread Dániel Urbán
Bumping - please consider voting on this KIP. TIA Daniel Dániel Urbán ezt írta (időpont: 2024. dec. 9., H, 9:06): > Gentle bump - please consider checking the KIP and voting. > Daniel > > Dániel Urbán ezt írta (időpont: 2024. dec. 5., > Cs, 12:08): > >> Bumping this

Re: [VOTE] KIP-1109: Unifying Kafka Consumer Topic Metrics

2024-12-12 Thread Chia-Ping Tsai
+1 (binding) Apoorv Mittal 於 2024年12月13日 週五 上午8:26寫道: > Hi All, > I would like to start vote on KIP-1109: > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-1109%3A+Unifying+Kafka+Consumer+Topic+Metrics > > Discussion thread: > https://li

[VOTE] KIP-1109: Unifying Kafka Consumer Topic Metrics

2024-12-12 Thread Apoorv Mittal
Hi All, I would like to start vote on KIP-1109: https://cwiki.apache.org/confluence/display/KAFKA/KIP-1109%3A+Unifying+Kafka+Consumer+Topic+Metrics Discussion thread: https://lists.apache.org/thread/fvn9r9lw55lyf971k6z4qnpnmq9tmm50 Regards, Apoorv Mittal

[RESULT] [VOTE] Release Kafka version 3.7.2

2024-12-12 Thread Matthias J. Sax
This vote passes with 8 +1 votes (4 bindings) and no 0 or -1 votes. +1 votes PMC Members: * Bill Bejeck * Divij Vaidya * Luke Chen * Matthias J. Sax Committers: * Andrew Schofield Community: * Jiunn-Yang * TengYao * Federico Valeri 0 votes * No votes -1 votes * No votes Vote thread: https

Re: [VOTE] 3.7.2 RC1

2024-12-12 Thread Matthias J. Sax
Thanks all for verifying the RC and for voting. I am closing the vote now as accepted. Will send a follow up email with the result, and move forward with the release. Also: +1 (binding) from my side -Matthias On 12/8/24 2:47 AM, Federico Valeri wrote: Hi, thanks for running the release

Re: [VOTE] KIP-1111: Enforcing Explicit Naming for Kafka Streams Internal Topics

2024-12-12 Thread Matthias J. Sax
(binding) Many Thanks! Bruno On 06.12.24 10:13, Lucas Brutschy wrote: Thanks for the KIP! +1 (binding) Cheers, Lucas On Fri, Dec 6, 2024 at 9:41 AM Sebastien Viale wrote: Hi, I'd like to start the vote on KIP- - Enforcing Explicit Naming for Kafka Streams Internal Topics:

Re: [VOTE] KIP-1111: Enforcing Explicit Naming for Kafka Streams Internal Topics

2024-12-11 Thread Sophie Blee-Goldman
gt; > > > +1 (binding) > > > > Many Thanks! > > Bruno > > > > On 06.12.24 10:13, Lucas Brutschy wrote: > > > Thanks for the KIP! > > > > > > +1 (binding) > > > > > > Cheers, > > > Lucas > > > > > >

Re: [VOTE] KIP-1034: Dead letter queue in Kafka Streams

2024-12-11 Thread Bruno Cadonna
De : Matthias J. Sax Envoyé : samedi 7 décembre 2024 00:39 À : dev@kafka.apache.org Objet : [EXT] Re: [VOTE] KIP-1034: Dead letter queue in Kafka Streams Warning External sender Do not click on any links or open any attachments unless you trust the sender and

Re: [VOTE] KIP-1111: Enforcing Explicit Naming for Kafka Streams Internal Topics

2024-12-10 Thread Bill Bejeck
gt; +1 (binding) > > > > Cheers, > > Lucas > > > > On Fri, Dec 6, 2024 at 9:41 AM Sebastien Viale > > wrote: > >> > >> Hi, > >> > >> I'd like to start the vote on KIP- - Enforcing Explicit Naming for > Kafka Streams Inte

Re: [VOTE]: KIP-1071: Streams Rebalance Protocol

2024-12-09 Thread Sophie Blee-Goldman
and Lucas, > Thanks for proposing this excellent KIP. > > +1 (binding) > > Thanks, > Andrew > > From: Alieh Saeedi > Sent: 03 December 2024 16:07 > To: dev@kafka.apache.org > Subject: Re: [VOTE]: KIP-1071: Streams Rebalance Pr

RE: Re: [VOTE]: KIP-1071: Streams Rebalance Protocol

2024-12-09 Thread Colt McNealy
; Thanks, > Andrew > > From: Alieh Saeedi > Sent: 03 December 2024 16:07 > To: dev@kafka.apache.org > Subject: Re: [VOTE]: KIP-1071: Streams Rebalance Protocol > > Thanks for the effort. I really appreciate the time and dedication you put > into this. >

Re: [VOTE] KIP-1034: Dead letter queue in Kafka Streams

2024-12-09 Thread Sebastien Viale
also propose nesting the class into the interface, as it is done for enums. cheers ! Sébastien De : Matthias J. Sax Envoyé : samedi 7 décembre 2024 00:39 À : dev@kafka.apache.org Objet : [EXT] Re: [VOTE] KIP-1034: Dead letter queue in Kafka Streams Warning

Re: [VOTE] KIP-1098: Reverse Checkpointing in MirrorMaker

2024-12-09 Thread Dániel Urbán
Gentle bump - please consider checking the KIP and voting. Daniel Dániel Urbán ezt írta (időpont: 2024. dec. 5., Cs, 12:08): > Bumping this vote - the change has a relatively small footprint, but fills > a sizable gap in MM2. > Please consider checking the KIP and chiming in. >

[VOTE] KIP-1118: Add Deadlock Protection on Producer Network Thread

2024-12-08 Thread TengYao Chi
Hi everyone, Based on our discussion <https://lists.apache.org/thread/8npgn4d7wwlwvdy1h4xpbdlffksstddl> regarding KIP-1118 <https://cwiki.apache.org/confluence/display/KAFKA/KIP-1118%3A+Add+Deadlock+Protection+on+Producer+Network+Thread>, I believe this KIP is now ready for a vote

Re: [VOTE] 3.7.2 RC1

2024-12-08 Thread Federico Valeri
nning the release. > > > > > > I did the following: > > > - Browsed the javadoc > > > - Browsed the protocol documentation > > > - Ran the quickstart with ZK > > > > > > +1 (non-binding) > > > > > > Thanks, > > &

Re: [VOTE] 3.7.2 RC1

2024-12-07 Thread TengYao Chi
ntation > > - Ran the quickstart with ZK > > > > +1 (non-binding) > > > > Thanks, > > Andrew > > > > From: Luke Chen > > Sent: 06 December 2024 07:22 > > To: dev@kafka.apache.org > > Cc: us...@kafka.apache.org ; kaf

Re: [VOTE] 3.7.2 RC1

2024-12-06 Thread 黃竣陽
ber 2024 07:22 > To: dev@kafka.apache.org > Cc: us...@kafka.apache.org ; kafka-clients > > Subject: Re: [VOTE] 3.7.2 RC1 > > Hi Matthias, > > Thanks for running the release. > > I did the following steps: > > - Validated the checksums and keys >

Re: [VOTE] KIP-1034: Dead letter queue in Kafka Streams

2024-12-06 Thread Matthias J. Sax
e fields and methods from the enums and move them into their respective classes: - ProductionExceptionHandler - DeserializationExceptionHandler - ProcessingExceptionHandler We have updated the KIP accordingly and are now calling for a new vote to maintain transparency. Cheers, Loïc, Damien, and Sé

Re: [VOTE]: KIP-1071: Streams Rebalance Protocol

2024-12-06 Thread Andrew Schofield
Hi Bruno and Lucas, Thanks for proposing this excellent KIP. +1 (binding) Thanks, Andrew From: Alieh Saeedi Sent: 03 December 2024 16:07 To: dev@kafka.apache.org Subject: Re: [VOTE]: KIP-1071: Streams Rebalance Protocol Thanks for the effort. I really

Re: [VOTE] KIP-1111: Enforcing Explicit Naming for Kafka Streams Internal Topics

2024-12-06 Thread Bruno Cadonna
Hi Sébastien, +1 (binding) Many Thanks! Bruno On 06.12.24 10:13, Lucas Brutschy wrote: Thanks for the KIP! +1 (binding) Cheers, Lucas On Fri, Dec 6, 2024 at 9:41 AM Sebastien Viale wrote: Hi, I'd like to start the vote on KIP- - Enforcing Explicit Naming for Kafka Streams Int

Re: [VOTE] KIP-1111: Enforcing Explicit Naming for Kafka Streams Internal Topics

2024-12-06 Thread Lucas Brutschy
Thanks for the KIP! +1 (binding) Cheers, Lucas On Fri, Dec 6, 2024 at 9:41 AM Sebastien Viale wrote: > > Hi, > > I'd like to start the vote on KIP- - Enforcing Explicit Naming for Kafka > Streams Internal Topics: > > https://cwiki.apache.org/confluence/display/K

Re: [VOTE] 3.7.2 RC1

2024-12-06 Thread Andrew Schofield
@kafka.apache.org Cc: us...@kafka.apache.org ; kafka-clients Subject: Re: [VOTE] 3.7.2 RC1 Hi Matthias, Thanks for running the release. I did the following steps: - Validated the checksums and keys - Browsed the javadocs - Browsed the documentation - Ran all the quickstart using KRaft with

[VOTE] KIP-1111: Enforcing Explicit Naming for Kafka Streams Internal Topics

2024-12-06 Thread Sebastien Viale
Hi, I'd like to start the vote on KIP- - Enforcing Explicit Naming for Kafka Streams Internal Topics: https://cwiki.apache.org/confluence/display/KAFKA/KIP-%3A+Enforcing+Explicit+Naming+for+Kafka+Streams+Internal+Topics This proposal aims to add a configuration that prevents a

Re: [VOTE] 3.7.2 RC1

2024-12-05 Thread Luke Chen
. Do we want to backport changes to 3.7 such as > > https://issues.apache.org/jira/browse/KAFKA-18051 and > > https://issues.apache.org/jira/browse/KAFKA-17062? The former is a > > good-to-have validation and the latter impacts data stored by Tiered > > Storage whic

Re: [VOTE] 3.7.2 RC1

2024-12-05 Thread Matthias J. Sax
end up creating one for a different reason, let's backport these. ## Vote + 1 (binding) -- Divij Vaidya On Thu, Dec 5, 2024 at 7:16 PM Josep Prat wrote: Hi Matthias, For 3.8.1 I pointed to the PR that makes the update to the new patch version. Best, -- Josep Prat

Re: [VOTE] 3.7.2 RC1

2024-12-05 Thread Matthias J. Sax
E_NOTES.html Please download, test and vote by Tuesday, Dec 10, 9am PST. Kafka's KEYS file containing PGP keys we use to sign the release: https://kafka.apache.org/KEYS * Release artifacts to be voted upon (source and binary): https://dist.apache.org/repos/dist/dev/kafka/3.7.2-rc1/ * Do

Re: [VOTE] 3.7.2 RC1

2024-12-05 Thread Divij Vaidya
/KAFKA-17062? The former is a good-to-have validation and the latter impacts data stored by Tiered Storage which is not GA in 3.7. Hence, we do not need a new RC, but if we do end up creating one for a different reason, let's backport these. ## Vote + 1 (binding) -- Divij Vaidya On Thu,

Re: [VOTE] 3.7.2 RC1

2024-12-05 Thread Josep Prat
gt;> > >> > >> This is the second candidate for release of Apache Kafka 3.7.2. This is > >> a bug-fix release, closing 21 Jira tickets. > >> > >> > >> > >> Release notes for the 3.7.2 release: > >> > https://dist.apache.org/repos

Re: [VOTE] KIP-1034: Dead letter queue in Kafka Streams

2024-12-05 Thread Lucas Brutschy
decided to remove mutable fields and methods > from the enums and move them into their respective classes: > > - ProductionExceptionHandler > - DeserializationExceptionHandler > - ProcessingExceptionHandler > > We have updated the KIP accordingly and are now calling for a new

Re: [VOTE] KIP-1098: Reverse Checkpointing in MirrorMaker

2024-12-05 Thread Dániel Urbán
Bumping this vote - the change has a relatively small footprint, but fills a sizable gap in MM2. Please consider checking the KIP and chiming in. TIA Daniel Viktor Somogyi-Vass ezt írta (időpont: 2024. dec. 2., H, 10:40): > +1 (binding) > > Thanks for the KIP Daniel! > > Viktor

Re: [VOTE] 3.7.2 RC1

2024-12-04 Thread Bill Bejeck
lease: > https://dist.apache.org/repos/dist/dev/kafka/3.7.2-rc1/RELEASE_NOTES.html > > Please download, test and vote by Tuesday, Dec 10, 9am PST. > > Kafka's KEYS file containing PGP keys we use to sign the release: > https://kafka.apache.org/KEYS > > > >

Re: [VOTE] KIP-1034: Dead letter queue in Kafka Streams

2024-12-04 Thread Sebastien Viale
the KIP accordingly and are now calling for a new vote to maintain transparency. Cheers, Loïc, Damien, and Sébastien De : Damien Gasparina Envoyé : vendredi 27 septembre 2024 09:06 À : dev@kafka.apache.org Objet : [EXT] Re: [VOTE] KIP-1034: Dead letter queue in

[VOTE] 3.7.2 RC1

2024-12-04 Thread Matthias J. Sax
download, test and vote by Tuesday, Dec 10, 9am PST. Kafka's KEYS file containing PGP keys we use to sign the release: https://kafka.apache.org/KEYS * Release artifacts to be voted upon (source and binary): https://dist.apache.org/repos/dist/dev/kafka/3.7.2-rc1/ * Docker release artifacts

[VOTE] [CANCELED] 3.7.2 RC0

2024-12-04 Thread Matthias J. Sax
need to bump the RC number. Just want to avoid confusion why I will call for a first vote for RC1 instead of RC0. -Matthias

[VOTE] [CANCELED] 3.7.2 RC0

2024-12-04 Thread Matthias J. Sax
need to bump the RC number. Just want to avoid confusion why I will call for a first vote for RC1 instead of RC0. -Matthias

Re: [VOTE]: KIP-1071: Streams Rebalance Protocol

2024-12-03 Thread Alieh Saeedi
Thanks for the effort. I really appreciate the time and dedication you put into this. +1(non-binding) Cheers, Alieh On Wed, Nov 27, 2024 at 11:31 AM Bruno Cadonna wrote: > Hi all, > > We (Lucas and myself) would like to call for a vote on KIP-1071: Streams > Rebalance Pro

Re: [VOTE] KIP-1099: Extend kafka-consumer-groups command line tool to support new consumer group

2024-12-03 Thread PoAn Yang
gt;>> +1 (binding) >>>> >>>> Jeff >>>> >>>> On 2024/11/25 15:46:04 "Lianet M." wrote: >>>>> Thanks for the KIP and updates PoAn! >>>>> >>>>> +1 (binding) >>>>> >>>

Re: [VOTE] KIP-655: Add deduplication processor in kafka-streams

2024-12-02 Thread Matthias J. Sax
. Nov. 2024, 13:40: Hi everyone, I would like to start a vote on KIP-655 which adds deduplication processor in kafka-streams: https://cwiki.apache.org/confluence/display/KAFKA/KIP-655%3A+Windowed+Distinct+Operation+for+Kafka+Streams+API Thanks, Ayoub

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

2024-12-02 Thread Swikar Patel
21 AM Divij Vaidya >> wrote: >> >>> Corresponding discussion thread: >>> https://lists.apache.org/thread/kxw9kxn49kt30kwckzwp8vq3k7kmrz9m >>> >>> Please vote if you agree with proceeding ahead with the proposal. Votes >>> from all community members are cordially invited. >>> >>> -- >>> Divij Vaidya >>> >>

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

2024-12-02 Thread Divij Vaidya
e KIP. +1 > > Jun > > On Wed, Nov 27, 2024 at 8:21 AM Divij Vaidya > wrote: > > > Corresponding discussion thread: > > https://lists.apache.org/thread/kxw9kxn49kt30kwckzwp8vq3k7kmrz9m > > > > Please vote if you agree with proceeding ahead with the proposal.

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

2024-12-02 Thread Jun Rao
Hi, Divij, Thanks for the KIP. +1 Jun On Wed, Nov 27, 2024 at 8:21 AM Divij Vaidya wrote: > Corresponding discussion thread: > https://lists.apache.org/thread/kxw9kxn49kt30kwckzwp8vq3k7kmrz9m > > Please vote if you agree with proceeding ahead with the proposal. Votes > fro

Re: [VOTE] KIP-1098: Reverse Checkpointing in MirrorMaker

2024-12-02 Thread Viktor Somogyi-Vass
+1 (binding) Thanks for the KIP Daniel! Viktor On Mon, Dec 2, 2024 at 10:36 AM Vidor Kanalas wrote: > Hi, thanks for the KIP! > +1 (non-binding) > > Best, > Vidor > On Mon, Dec 2, 2024 at 10:15 AM Dániel Urbán > wrote: > > > Hi everyone, > > > &

Re: [VOTE] KIP-1098: Reverse Checkpointing in MirrorMaker

2024-12-02 Thread Vidor Kanalas
Hi, thanks for the KIP! +1 (non-binding) Best, Vidor On Mon, Dec 2, 2024 at 10:15 AM Dániel Urbán wrote: > Hi everyone, > > I'd like to start the vote on KIP-1098: Reverse Checkpointing in > MirrorMaker ( > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-1098%3A+R

[VOTE] KIP-1098: Reverse Checkpointing in MirrorMaker

2024-12-02 Thread Dániel Urbán
Hi everyone, I'd like to start the vote on KIP-1098: Reverse Checkpointing in MirrorMaker ( https://cwiki.apache.org/confluence/display/KAFKA/KIP-1098%3A+Reverse+Checkpointing+in+MirrorMaker ). TIA, Daniel

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

2024-11-28 Thread Satish Duggana
Divij Vaidya > > > wrote: > > > > > > > Corresponding discussion thread: > > > > https://lists.apache.org/thread/kxw9kxn49kt30kwckzwp8vq3k7kmrz9m > > > > > > > > Please vote if you agree with proceeding ahead with the proposal. Votes > > > > from all community members are cordially invited. > > > > > > > > -- > > > > Divij Vaidya > > > > > > >

Re: [VOTE] KIP-1099: Extend kafka-consumer-groups command line tool to support new consumer group

2024-11-28 Thread PoAn Yang
Lianet >>>> >>>> On Mon, Nov 25, 2024, 10:43 a.m. Andrew Schofield < >>>> andrew_schofield_j...@outlook.com> wrote: >>>> >>>>> +1 (non-binding) >>>>> >>>>> Thanks, >>>>> Andrew >&g

Re: [VOTE] KIP-1099: Extend kafka-consumer-groups command line tool to support new consumer group

2024-11-28 Thread Lucas Brutschy
t;> On Mon, Nov 25, 2024, 10:43 a.m. Andrew Schofield < > >> andrew_schofield_j...@outlook.com> wrote: > >> > >>> +1 (non-binding) > >>> > >>> Thanks, > >>> Andrew > >>> > >>> ___

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

2024-11-28 Thread Federico Valeri
gt; Thanks for the KIP! > > > > Luke > > > > On Thu, Nov 28, 2024 at 12:21 AM Divij Vaidya > > wrote: > > > > > Corresponding discussion thread: > > > https://lists.apache.org/thread/kxw9kxn49kt30kwckzwp8vq3k7kmrz9m > > > > >

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

2024-11-27 Thread Kamal Chandraprakash
thread: > > https://lists.apache.org/thread/kxw9kxn49kt30kwckzwp8vq3k7kmrz9m > > > > Please vote if you agree with proceeding ahead with the proposal. Votes > > from all community members are cordially invited. > > > > -- > > Divij Vaidya > > >

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

2024-11-27 Thread Luke Chen
Hi Divij, +1 (binding) from me. Thanks for the KIP! Luke On Thu, Nov 28, 2024 at 12:21 AM Divij Vaidya wrote: > Corresponding discussion thread: > https://lists.apache.org/thread/kxw9kxn49kt30kwckzwp8vq3k7kmrz9m > > Please vote if you agree with proceeding ahead with the pro

Re: [VOTE]: KIP-1071: Streams Rebalance Protocol

2024-11-27 Thread Bill Bejeck
Thanks for the KIP Bruno and Lucas! This is a great addition to Kafka Streams. +1(binding) Regards, Bill On Wed, Nov 27, 2024 at 5:31 AM Bruno Cadonna wrote: > Hi all, > > We (Lucas and myself) would like to call for a vote on KIP-1071: Streams > Rebalance Protocol: https://cwik

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

2024-11-27 Thread Divij Vaidya
Corresponding discussion thread: https://lists.apache.org/thread/kxw9kxn49kt30kwckzwp8vq3k7kmrz9m Please vote if you agree with proceeding ahead with the proposal. Votes from all community members are cordially invited. -- Divij Vaidya

  1   2   3   4   5   6   7   8   9   10   >