Re: [VOTE] KIP-633: Drop 24 hour default of grace period in Streams

2021-04-08 Thread Sophie Blee-Goldman
Hey all, This KIP has been accepted with four +1 (binding) votes from John, Guozhang, Matthias, and myself four +1 (non-binding) votes from Leah, Walker, Lotz, and Israel Thanks everyone. Israel will take it from here On Thu, Apr 8, 2021 at 2:58 PM Sophie Blee-Goldman wrote: > > I would also

Re: [DISCUSS] KIP-726: Make the CooperativeStickyAssignor as the default assignor

2021-04-08 Thread Sophie Blee-Goldman
Alright, here's the detailed proposal for KAFKA-12477. This assumes we will change the default assignor to ["cooperative-sticky", "range"] in KIP-726. It also acknowledges that users may attempt any kind of upgrade without reading the docs, and so we need to put in safeguards against data

Re: [VOTE] KIP-725: Streamlining configurations for TimeWindowedDeserializer.

2021-04-08 Thread Sagar
Thanks for the very precise time John :D I think that time has been passed so I am going to close the voting. Thanks! Sagar. On Fri, Apr 9, 2021 at 12:28 AM John Roesler wrote: > Hi Sagar, > > It does have three binding votes, but it looks like it's a > little shy of the mandatory 72 hours

[jira] [Created] (KAFKA-12638) Remove default implementation of ConsumerRebalanceListener#onPartitionsLost

2021-04-08 Thread A. Sophie Blee-Goldman (Jira)
A. Sophie Blee-Goldman created KAFKA-12638: -- Summary: Remove default implementation of ConsumerRebalanceListener#onPartitionsLost Key: KAFKA-12638 URL: https://issues.apache.org/jira/browse/KAFKA-12638

[jira] [Created] (KAFKA-12637) Remove deprecated PartitionAssignor interface

2021-04-08 Thread A. Sophie Blee-Goldman (Jira)
A. Sophie Blee-Goldman created KAFKA-12637: -- Summary: Remove deprecated PartitionAssignor interface Key: KAFKA-12637 URL: https://issues.apache.org/jira/browse/KAFKA-12637 Project: Kafka

Jenkins build is unstable: Kafka » Kafka Branch Builder » trunk #20

2021-04-08 Thread Apache Jenkins Server
See

Re: [VOTE] 2.6.2 RC1

2021-04-08 Thread Guozhang Wang
Looked over the javadocs and web docs again. Download the jars and check the license files are all updated now (thanks to John again!). +1 On Thu, Apr 8, 2021 at 1:48 PM Sophie Blee-Goldman wrote: > Hello Kafka users, developers and client-developers, > > This is the second candidate for

[jira] [Created] (KAFKA-12636) Ensure retention still enforced for compacted topics if cleaning is not enabled

2021-04-08 Thread Jason Gustafson (Jira)
Jason Gustafson created KAFKA-12636: --- Summary: Ensure retention still enforced for compacted topics if cleaning is not enabled Key: KAFKA-12636 URL: https://issues.apache.org/jira/browse/KAFKA-12636

Jenkins build is back to normal : Kafka » kafka-2.7-jdk8 #144

2021-04-08 Thread Apache Jenkins Server
See

Re: [VOTE] KIP-633: Drop 24 hour default of grace period in Streams

2021-04-08 Thread Sophie Blee-Goldman
> I would also like to volunteer to implement it if that is ok. That would be awesome -- I've been pretty overbooked lately and was literally just about to ask for volunteers to take over this KIP. Perfect timing :) The KIP still has about 4 hours to go on the voting but it looks like it'll

Re: [VOTE] KIP-633: Drop 24 hour default of grace period in Streams

2021-04-08 Thread Israel Ekpo
I have reviewed the KIP. The motivation makes sense and the recommended API changes make sense as well. https://cwiki.apache.org/confluence/display/KAFKA/KIP-633%3A+Drop+24+hour+default+of+grace+period+in+Streams So +1 I would also like to volunteer to implement it if that is ok. On Thu, Apr

Re: [DISCUSS] KIP-633: Drop 24 hour default of grace period in Streams

2021-04-08 Thread Matthias J. Sax
Thanks! On 4/8/21 2:06 PM, Sophie Blee-Goldman wrote: > 1) Since the new APIs (eg ofSizeWithNoGrace and ofSizeAndGrace) are the > only static constructors > after this change, there seems to be no reason to keep the .grace around -- > you've already specified > it with your choice of the static

Re: [VOTE] KIP-633: Drop 24 hour default of grace period in Streams

2021-04-08 Thread Matthias J. Sax
+1 (binding) On 4/6/21 10:15 AM, Lotz Utfpr wrote: > Makes sense to me! +1 > > Apologies for being brief. This email was sent from my mobile phone. > >> On 6 Apr 2021, at 18:45, Walker Carlson >> wrote: >> >> This makes sense to me +1! >> >> Walker >> >>> On Tue, Apr 6, 2021 at 11:08 AM

Re: [DISCUSS] KIP-730: Producer ID generation in KRaft mode

2021-04-08 Thread Ron Dagostino
Hi David. I'm wondering if it might be a good idea to have the broker send information about the last block it successfully received when it requests a new block. As the RPC stands right now it can't be idempotent -- it just tells the controller "provide me a new block, please". One case where

Re: [DISCUSS] KIP-633: Drop 24 hour default of grace period in Streams

2021-04-08 Thread Sophie Blee-Goldman
1) Since the new APIs (eg ofSizeWithNoGrace and ofSizeAndGrace) are the only static constructors after this change, there seems to be no reason to keep the .grace around -- you've already specified it with your choice of the static constructor. 2) Ack, updated the KIP 3) Ack, fixed On Tue, Apr

Re: [VOTE] 2.6.2 RC0

2021-04-08 Thread Sophie Blee-Goldman
Closing this vote, please refer to the [VOTE] 2.6.2 RC1 thread On Wed, Mar 31, 2021 at 3:41 PM Justin Mclean wrote: > Hi, > > > Can you clarify a few things? > > Sorry I'm not subscribed to this list and only just saw this. > > > - On the first point, the only thing I see is zstd, which we do

[VOTE] 2.6.2 RC1

2021-04-08 Thread Sophie Blee-Goldman
Hello Kafka users, developers and client-developers, This is the second candidate for release of Apache Kafka 2.6.2. Apache Kafka 2.6.2 is a bugfix release and fixes 35 issues since the 2.6.1 release. Please see the release notes for more information. Release notes for the 2.6.2 release:

Jenkins build is still unstable: Kafka » Kafka Branch Builder » 2.8 #5

2021-04-08 Thread Apache Jenkins Server
See

[jira] [Created] (KAFKA-12635) Mirrormaker 2 offset sync is incorrect if the target partition is empty

2021-04-08 Thread Frank Yi (Jira)
Frank Yi created KAFKA-12635: Summary: Mirrormaker 2 offset sync is incorrect if the target partition is empty Key: KAFKA-12635 URL: https://issues.apache.org/jira/browse/KAFKA-12635 Project: Kafka

Re: [DISCUSS] KIP-730: Producer ID generation in KRaft mode

2021-04-08 Thread Ron Dagostino
Oh, I see. Yes, my mistake -- I read it wrong. You are right that all we need in the metadata log is the latest value allocated. Ron On Thu, Apr 8, 2021 at 11:21 AM David Arthur wrote: > > Ron -- I considered making the RPC response and record use the same (or > very similar) fields, but the

Re: [VOTE] KIP-725: Streamlining configurations for TimeWindowedDeserializer.

2021-04-08 Thread John Roesler
Hi Sagar, It does have three binding votes, but it looks like it's a little shy of the mandatory 72 hours you have to leave the vote open: https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals#KafkaImprovementProposals-Process Assuming no one shows up with a veto, you can

Re: [VOTE] KIP-725: Streamlining configurations for TimeWindowedDeserializer.

2021-04-08 Thread Sagar
Thanks Sophie/ John/ Leah and Guozhang. Can I assume that this kip has received sufficient votes to be marked as accepted ? Sagar. On Tue, 6 Apr 2021 at 9:38 PM, Guozhang Wang wrote: > +1. Thanks! > > On Tue, Apr 6, 2021 at 7:01 AM Leah Thomas > wrote: > > > Hi Sagar, +1 non-binding. Thanks

[jira] [Resolved] (KAFKA-12619) Ensure LeaderChange message is committed before initializing high watermark

2021-04-08 Thread Jason Gustafson (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12619?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson resolved KAFKA-12619. - Fix Version/s: 2.8.1 3.0.0 Resolution: Fixed > Ensure

[VOTE] 2.7.1 RC2

2021-04-08 Thread Mickael Maison
Hello Kafka users, developers and client-developers, This is the third candidate for release of Apache Kafka 2.7.1. Since 2.7.1 RC1, the following JIRAs have been fixed: KAFKA-12593, KAFKA-12474, KAFKA-12602. Release notes for the 2.7.1 release:

[jira] [Resolved] (KAFKA-12457) Implications of KIP-516 for quorum controller

2021-04-08 Thread Jason Gustafson (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12457?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson resolved KAFKA-12457. - Resolution: Fixed > Implications of KIP-516 for quorum controller >

Build failed in Jenkins: Kafka » kafka-2.7-jdk8 #143

2021-04-08 Thread Apache Jenkins Server
See Changes: [Mickael Maison] KAFKA-12602: Fix LICENSE file (#10474) -- [...truncated 6.91 MB...]

[jira] [Created] (KAFKA-12634) Should checkpoint after restore finished

2021-04-08 Thread Matthias J. Sax (Jira)
Matthias J. Sax created KAFKA-12634: --- Summary: Should checkpoint after restore finished Key: KAFKA-12634 URL: https://issues.apache.org/jira/browse/KAFKA-12634 Project: Kafka Issue Type:

[jira] [Created] (KAFKA-12633) Remove deprecated "TopologyTestDriver#pipeInput / readOutput"

2021-04-08 Thread Guozhang Wang (Jira)
Guozhang Wang created KAFKA-12633: - Summary: Remove deprecated "TopologyTestDriver#pipeInput / readOutput" Key: KAFKA-12633 URL: https://issues.apache.org/jira/browse/KAFKA-12633 Project: Kafka

[jira] [Resolved] (KAFKA-12630) Remove deprecated KafkaClientSupplier#getAdminClient

2021-04-08 Thread Guozhang Wang (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12630?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang resolved KAFKA-12630. --- Fix Version/s: 3.0.0 Resolution: Fixed > Remove deprecated

Re: [DISCUSS] KIP-730: Producer ID generation in KRaft mode

2021-04-08 Thread David Arthur
Ron -- I considered making the RPC response and record use the same (or very similar) fields, but the use case is slightly different. A broker handling the RPC needs to know the bounds of the block since it has no idea what the block size is. Also, the brokers will normally see non-contiguous

Re: I have a question about apache kafka.

2021-04-08 Thread John Roesler
Hello Yun Han Nam, The users@ list is more typical for this kind of topic, but this list is fine, too. What’s the question? Thanks, John On Thu, Apr 8, 2021, at 06:25, 남윤한[Yun Han Nam] wrote: > Hi. > > I want to ask you a question about Apache Kafka, is this the right one? > > It is a

I have a question about apache kafka.

2021-04-08 Thread 남윤한 [Yun Han Nam]
Hi. I want to ask you a question about Apache Kafka, is this the right one? It is a technical question about error log that comes too often.​

Re: [VOTE] 2.7.1 RC1

2021-04-08 Thread Mickael Maison
Hi, Closing this vote as we've had to address https://issues.apache.org/jira/browse/KAFKA-12602. I'll get a new RC out shortly Thanks On Fri, Mar 26, 2021 at 11:48 AM Mickael Maison wrote: > > Hello Kafka users, developers and client-developers, > > This is the second candidate for release of

Jenkins build is still unstable: Kafka » Kafka Branch Builder » trunk #18

2021-04-08 Thread Apache Jenkins Server
See

[jira] [Resolved] (KAFKA-12623) Fix LICENSE in 2.7

2021-04-08 Thread Mickael Maison (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12623?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Mickael Maison resolved KAFKA-12623. Resolution: Fixed > Fix LICENSE in 2.7 > -- > > Key:

Jenkins build is still unstable: Kafka » Kafka Branch Builder » trunk #17

2021-04-08 Thread Apache Jenkins Server
See

[jira] [Created] (KAFKA-12632) Exception may missed when source task failing

2021-04-08 Thread Jira
戈震 created KAFKA-12632: -- Summary: Exception may missed when source task failing Key: KAFKA-12632 URL: https://issues.apache.org/jira/browse/KAFKA-12632 Project: Kafka Issue Type: Bug

Re: [ANNOUNCE] New Kafka PMC Member: Bill Bejeck

2021-04-08 Thread Tom Bentley
Congratulations Bill! On Thu, Apr 8, 2021 at 2:36 AM Luke Chen wrote: > Congratulations Bill! > > Luke > > On Thu, Apr 8, 2021 at 9:17 AM Matthias J. Sax wrote: > > > Hi, > > > > It's my pleasure to announce that Bill Bejeck in now a member of the > > Kafka PMC. > > > > Bill has been a Kafka

Jenkins build is still unstable: Kafka » Kafka Branch Builder » trunk #16

2021-04-08 Thread Apache Jenkins Server
See