Re: [VOTE] KIP-590: Redirect Zookeeper Mutation Protocols to The Controller

2020-07-28 Thread David Jacot
Hi Boyang, Thanks for the update. 1./2. In KIP-599 (accepted and already in trunk), we throttle the CreateTopicsRequest, CreatePartitionsRequest, and DeleteTopicsRequests by muting the channel used by the Admin client and setting the throttleTimeMs in the response. The change that you propose bre

Re: [VOTE] KIP-607: Add Metrics to Record the Memory Used by RocksDB to Kafka Streams

2020-07-28 Thread Bruno Cadonna
Thank you for voting. The updated KIP-607 is accepted with - 3 binding votes (John, Bill, and Guozhang) - 1 non-binding vote (Navinder) Best, Bruno On 27.07.20 20:06, Guozhang Wang wrote: +1 (binding). Thanks. On Mon, Jul 27, 2020 at 9:24 AM Bill Bejeck wrote: Thanks for the KIP Bruno. +

Re: [DISCUSS] KIP-405: Kafka Tiered Storage

2020-07-28 Thread Satish Duggana
HI Jun, Thanks for your comments. We put our inline replies below. 1001. I was thinking that you could just use the tiered metadata to do the reconciliation. The tiered metadata contains offset ranges and epoch history. Those should be enough for reconciliation purposes. If we use remote storage

Build failed in Jenkins: kafka-2.4-jdk8 #235

2020-07-28 Thread Apache Jenkins Server
See Changes: [matthias] MINOR: Remove staticmethod tag to be able to use logger of instance -- [...truncated 8.37 MB...] org.apache.kafka.streams.test.OutputVerifierTest >

Re: [VOTE] KIP-599: Throttle Create Topic, Create Partition and Delete Topic Operations

2020-07-28 Thread David Jacot
Hi all, Just a quick update. We have made good progress regarding the implementation of this KIP. The major parts are already in trunk modulo the new "rate" implementation which is still under development. I would like to change the type of the `controller_mutations_rate` from a Long to a Double.

Re: [VOTE] KIP-635: GetOffsetShell: support for multiple topics and consumer configuration override

2020-07-28 Thread Viktor Somogyi-Vass
+1 from me (non-binding), thanks for the KIP. On Mon, Jul 27, 2020 at 10:02 AM Dániel Urbán wrote: > Hello everyone, > > I'd like to start a vote on KIP-635. The KIP enhances the GetOffsetShell > tool by enabling querying multiple topic-partitions, adding new filtering > options, and adding a co

[jira] [Created] (KAFKA-10318) Default API timeout must be enforced to be greater than request timeout just like in AdminClient

2020-07-28 Thread Gabor Somogyi (Jira)
Gabor Somogyi created KAFKA-10318: - Summary: Default API timeout must be enforced to be greater than request timeout just like in AdminClient Key: KAFKA-10318 URL: https://issues.apache.org/jira/browse/KAFKA-10318

[VOTE] KIP-450: Sliding Window Aggregations in the DSL

2020-07-28 Thread Leah Thomas
Hi all, I'd like to kick-off the vote for KIP-450 , adding sliding window aggregations to the DSL. The discussion thread is here

Request for access to create KIP

2020-07-28 Thread John Thomas
Hello Team, Could you please grant me access to Create KIP's. My ID is "johnthotekat" I'm a newbie stating to work on https://issues.apache.org/jira/browse/KAFKA-10316 Thank you! John

Re: Request for access to create KIP

2020-07-28 Thread John Thomas
johnthotekat From: John Thomas Sent: 27 July 2020 23:29 To: dev@kafka.apache.org Subject: Request for access to create KIP Hello Team, Could you please grant me access to Create KIP's. My ID is "johnthotekat" I'm a newbie stating to work on https://issues.apac

[jira] [Created] (KAFKA-10319) Fix unknown offset sum on trunk

2020-07-28 Thread John Roesler (Jira)
John Roesler created KAFKA-10319: Summary: Fix unknown offset sum on trunk Key: KAFKA-10319 URL: https://issues.apache.org/jira/browse/KAFKA-10319 Project: Kafka Issue Type: Sub-task

[jira] [Created] (KAFKA-10320) Log metrics for future logs never have the is-future tag removed

2020-07-28 Thread Bob Barrett (Jira)
Bob Barrett created KAFKA-10320: --- Summary: Log metrics for future logs never have the is-future tag removed Key: KAFKA-10320 URL: https://issues.apache.org/jira/browse/KAFKA-10320 Project: Kafka

Re: Request for access to create KIP

2020-07-28 Thread Boyang Chen
Permission granted On Tue, Jul 28, 2020 at 9:01 AM John Thomas wrote: > johnthotekat > > > From: John Thomas > Sent: 27 July 2020 23:29 > To: dev@kafka.apache.org > Subject: Request for access to create KIP > > Hello Team, > > Could you please grant me access t

Re: [DISCUSS] KIP-622 Add currentSystemTimeMs and currentStreamTimeMs to ProcessorContext

2020-07-28 Thread John Roesler
Thanks Matthias, This is a really good point. It might be a bummer to have to actually change the topology between testing and production. Do you think we can rather evolve the TimestampExtractor interface to let Streams pass in the current system time, along with the current record and the curren

Re: [DISCUSS] KIP-405: Kafka Tiered Storage

2020-07-28 Thread Jun Rao
Hi, Satish, Thanks for the reply. 1001. In your example, I was thinking that you could just download the latest leader epoch from the object store. After that you know the leader should end with offset 1100. The leader will delete all its local data before offset 1000 and start accepting new mess

Re: [DISCUSS] KIP-645: Replace abstract class Windows with a proper interface

2020-07-28 Thread John Roesler
Hi Sophie, A quick update: I've pushed a commit to the POC PR that includes the migration of Window to become a data class instead of an abstract class. It's quite a bit of code, but it does look like there is a clean deprecation/migration path. The basic idea is that we drop the "abstract" modif

how one can configure that consumer must consume topic messages by 150 mins (exactly 2.5 hours)

2020-07-28 Thread Rao, Vasudevan P
Hi I am working on Kafka in our environment. We need to restrict or delay the consumer to consume topic by 150 mins. I did some tweak in server. properties at the broker level (we have 5 brokers in one env) but that does not work. Namely I made changes in the servrer.properties under that gro

Re: [DISCUSS] KIP-405: Kafka Tiered Storage

2020-07-28 Thread Ying Zheng
1001. We did consider this approach. The concerns are 1) This makes unclean-leader-election rely on remote storage. In case the remote storage is unavailable, Kafka will not be able to finish the unclean-leader-election. 2) Since the user set local retention time (or local retention bytes), I th

Re: [DISCUSS] KIP-645: Replace abstract class Windows with a proper interface

2020-07-28 Thread Sophie Blee-Goldman
Awesome, thanks for looking into the Window improvements as well! (And I'm sorry I brought this upon you). I hope it's not too painful to get everything in the Windows ecosystem looking good and reasonable, and the benefits are pretty clear. Haven't had a careful look through the POC yet but the p

Re: [VOTE] KIP-590: Redirect Zookeeper Mutation Protocols to The Controller

2020-07-28 Thread Colin McCabe
Hi David, Thanks for bringing this up. This is indeed something that we overlooked, that we'll have to figure out. The active controller may not be co-located with a broker in the post-KIP-500 world. So it does not make sense to have the client communicate directly with the controller. Just

Re: [VOTE] KIP-590: Redirect Zookeeper Mutation Protocols to The Controller

2020-07-28 Thread Colin McCabe
Hi Boyang, Thanks for updating this. A few comments below: In the "Routing Request Security" section, there is a reference to "older requests that need redirection." But after these new revisions, both new and old requests need redirection. So we should rephrase this. > In addition, to avoi

Build failed in Jenkins: kafka-trunk-jdk14 #324

2020-07-28 Thread Apache Jenkins Server
See Changes: [github] KAFKA-10224: Update jersey license from CDDL to EPLv2 (#9089) -- [...truncated 3.20 MB...] org.apache.kafka.streams.TopologyTestDriverTest > should

[VOTE] 2.6.0 RC2

2020-07-28 Thread Randall Hauch
Hello Kafka users, developers and client-developers, This is the third candidate for release of Apache Kafka 2.6.0. This is a major release that includes many new features, including: * TLSv1.3 has been enabled by default for Java 11 or newer. * Smooth scaling out of Kafka Streams applications *

Jenkins build is back to normal : kafka-trunk-jdk11 #1675

2020-07-28 Thread Apache Jenkins Server
See

Re: [VOTE] KIP-554: Add Broker-side SCRAM Config API

2020-07-28 Thread Ron Dagostino
Hi everyone. I just wanted to notify that while implementing this I discovered that we had declared the ScramMechanism enum to have the values HMAC_SHA_{256,512} instead of SCRAM_SHA_{256,512}. I believe Rajini had indicated that this should be changed back on May 7th, and the change makes sense

Jenkins build is back to normal : kafka-2.3-jdk8 #221

2020-07-28 Thread Apache Jenkins Server
See

Re: [DISCUSS] KIP-450: Sliding Windows

2020-07-28 Thread Leah Thomas
Another minor tweak, instead of defining the window by the *size*, it will be defined by *timeDifference*, which is the maximum time difference between two events. This is a more precise way to define a window due to its inclusive ends, while allowing the user to create the window they expect. This

Re: [VOTE] 2.6.0 RC1

2020-07-28 Thread Randall Hauch
I've announced RC2 on a different thread entitled "[VOTE] 2.6.0 RC2" (see https://lists.apache.org/thread.html/rc8a3aa6986204adbb9ff326b8de849b3c8bac5b6b2b436e8143afea9%40%3Cdev.kafka.apache.org%3E). Please use that thread to highlight any blockers with that release candidate. Best regards, Randa

[VOTE] KIP-648: Renaming getter method for Interactive Queries

2020-07-28 Thread John Thomas
Hello everyone, I'd like to kick-off a vote for KIP-648 : Renaming getter method for Interactive Queries https://cwiki.apache.org/confluence/display/KAFKA/KIP-648%3A+Renaming+getter+method+for+Interactive+Queries It's a straight forward change to include new getters and deprecate the existing o

Re: [DISCUSS] KIP-450: Sliding Windows

2020-07-28 Thread Sophie Blee-Goldman
Thanks for the update Leah -- I think that all makes sense. Cheers, Sophie On Tue, Jul 28, 2020 at 3:55 PM Leah Thomas wrote: > Another minor tweak, instead of defining the window by the *size*, it will > be defined by *timeDifference*, which is the maximum time difference > between two events.

Re: [VOTE] KIP-450: Sliding Window Aggregations in the DSL

2020-07-28 Thread Sophie Blee-Goldman
Thanks for the KIP! It's been an enlightening discussion +1 (non-binding) Sophie On Tue, Jul 28, 2020 at 8:03 AM Leah Thomas wrote: > Hi all, > > I'd like to kick-off the vote for KIP-450 > < > https://cwiki.apache.org/confluence/display/KAFKA/KIP-450%3A+Sliding+Window+Aggregations+in+the+DSL

Re: [DISCUSS] KIP-450: Sliding Windows

2020-07-28 Thread Matthias J. Sax
Thanks Leah. Overall LGTM. A few nits: - the first figure shows window [9,19] but the window is not aligned properly (it should be 1ms to the right; right now, it's aligned to window [8,18]) - in the second figure, a hopping window would create more windows, ie, the first window would be [-6,14)

Re: [VOTE] KIP-450: Sliding Window Aggregations in the DSL

2020-07-28 Thread Matthias J. Sax
+1 (binding) On 7/28/20 4:35 PM, Sophie Blee-Goldman wrote: > Thanks for the KIP! It's been an enlightening discussion > > +1 (non-binding) > > Sophie > > On Tue, Jul 28, 2020 at 8:03 AM Leah Thomas wrote: > >> Hi all, >> >> I'd like to kick-off the vote for KIP-450 >> < >> https://cwiki.apac

Re: [VOTE] KIP-450: Sliding Window Aggregations in the DSL

2020-07-28 Thread Guozhang Wang
+1 (binding) On Tue, Jul 28, 2020 at 4:44 PM Matthias J. Sax wrote: > +1 (binding) > > On 7/28/20 4:35 PM, Sophie Blee-Goldman wrote: > > Thanks for the KIP! It's been an enlightening discussion > > > > +1 (non-binding) > > > > Sophie > > > > On Tue, Jul 28, 2020 at 8:03 AM Leah Thomas > wrote:

Build failed in Jenkins: kafka-2.6-jdk8 #102

2020-07-28 Thread Apache Jenkins Server
See Changes: [rhauch] KAFKA-10224: Update jersey license from CDDL to EPLv2 (#9089) -- [...truncated 6.30 MB...] org.apache.kafka.streams.test.ConsumerRecordFactoryTest > s

Jenkins build is back to normal : kafka-2.5-jdk8 #173

2020-07-28 Thread Apache Jenkins Server
See

Re: Re: [Discuss] KIP-581: Value of optional null field which has default value

2020-07-28 Thread 379377...@qq.com
Hi Chris, Thanks for your good suggestion, the KIP document and draft PR has been updated, please review again. And I found due to my misoperation, the mail thread has been broken, no idea how to fix it. Thanks Cheng Pan From: Christopher Egerton Date: 2020-05-04 10:53 To: dev Subject: Re

Re: [DISCUSS] KIP-630: Kafka Raft Snapshot

2020-07-28 Thread Jose Garcia Sancio
Thanks Ron. Your comments and suggestions were helpful. You can see my changes to the KIP here: https://cwiki.apache.org/confluence/pages/diffpagesbyversion.action?pageId=158864763&selectedPageVersions=15&selectedPageVersions=14 My comments are below... On Mon, Jul 27, 2020 at 11:29 AM Ron Dagost

Re: [VOTE] KIP-590: Redirect Zookeeper Mutation Protocols to The Controller

2020-07-28 Thread Boyang Chen
Thanks for the feedback Colin! On Tue, Jul 28, 2020 at 2:11 PM Colin McCabe wrote: > Hi Boyang, > > Thanks for updating this. A few comments below: > > In the "Routing Request Security" section, there is a reference to "older > requests that need redirection." But after these new revisions, bo

Build failed in Jenkins: kafka-trunk-jdk8 #4749

2020-07-28 Thread Apache Jenkins Server
See Changes: [github] KAFKA-10224: Update jersey license from CDDL to EPLv2 (#9089) -- [...truncated 2.76 MB...] org.apache.kafka.streams.integration.KTableKTableForeignK

Re: Build failed in Jenkins: kafka-trunk-jdk8 #4749

2020-07-28 Thread Benny Lee
From: Apache Jenkins Server Sent: Wednesday, 29 July 2020 2:12 PM To: dev@kafka.apache.org Subject: Build failed in Jenkins: kafka-trunk-jdk8 #4749 See Changes: [github] KAFK

Jenkins build is back to normal : kafka-2.4-jdk8 #236

2020-07-28 Thread Apache Jenkins Server
See

[jira] [Created] (KAFKA-10321) shouldDieOnInvalidOffsetExceptionWhileRunning would block forever on JDK11

2020-07-28 Thread Boyang Chen (Jira)
Boyang Chen created KAFKA-10321: --- Summary: shouldDieOnInvalidOffsetExceptionWhileRunning would block forever on JDK11 Key: KAFKA-10321 URL: https://issues.apache.org/jira/browse/KAFKA-10321 Project: Kaf