Re: [VOTE] KIP-345: Introduce static membership protocol to reduce consumer rebalances

2018-12-10 Thread Boyang Chen
Thanks Stanislav! Get Outlook for iOS From: Stanislav Kozlovski Sent: Monday, December 10, 2018 11:28 PM To: dev@kafka.apache.org Subject: Re: [VOTE] KIP-345: Introduce static membership protocol to reduce consumer rebalances This is

Re: [DISCUSS] KIP-373: Allow users to create delegation tokens for other users

2018-12-10 Thread Manikumar
Hi Harsha, Thanks for the review. With this KIP a designated superuser can create tokens without requiring individual user credentials. Any client can authenticate brokers using the created tokens. We may not call this as impersonation, since the clients API calls are executing on their own

Re: [VOTE] KIP-345: Introduce static membership protocol to reduce consumer rebalances

2018-12-10 Thread Stanislav Kozlovski
This is great work, Boyang. Thank you very much. +1 (non-binding) On Mon, Dec 10, 2018 at 6:09 PM Boyang Chen wrote: > Hey there, could I get more votes on this thread? > > Thanks for the vote from Mayuresh and Mike :) > > Best, > Boyang > > From: Mayuresh

Re: [DISCUSS] KIP-395: Encypt-then-MAC Delegation token metadata

2018-12-10 Thread Manikumar
Hi, Thanks for the KIP. Currently, master/secret key is stored as plain text in server.properties config file. Using master secret key as shared secret is again a security risk. We have raised KAFKA-7694 to implement a ZooKeeper based master/secret key management to automate secret key rotation.

Re: [DISCUSS] KIP-382: MirrorMaker 2.0

2018-12-10 Thread Dong Lin
Hey Ryanne, Thanks much for the KIP! Though I don't have time to review this KIP in detail at this stage, I think this KIP will be very useful to Apache Kafka users (particularly global enterprise users) who need geo replication capability. Currently Kafka users have to setup and manage MM

Re: Vote for KIP-393 (Fix time windowed serde to deserialize changelog topic)

2018-12-10 Thread Shawn Nguyen
Thanks for the feedback Guozhang! I updated the KIP. In the meantime, could I ask for additional binding votes/approval on this KIP proposal? Shawn On Thu, Dec 6, 2018 at 1:22 PM Liquan Pei wrote: > +1 (non-binding) > > On Wed, Dec 5, 2018 at 4:51 PM Guozhang Wang wrote: > >> Hello Shawn, >>

Build failed in Jenkins: kafka-2.1-jdk8 #79

2018-12-10 Thread Apache Jenkins Server
See Changes: [wangguoz] KAFKA-7610; Proactively timeout new group members if rebalance is -- [...truncated 912.63 KB...] kafka.zk.ReassignPartitionsZNodeTest >

Re: [DISCUSS] KIP-401 TransformerSupplier/ProcessorSupplier enhancements

2018-12-10 Thread Paul Whalen
Ah yes of course, this was an oversight, I completely ignored the multiple processors sharing the same state store when writing up the KIP. Which is funny, because I've actually done this (different processors sharing state stores) a fair amount myself, and I've settled on a pattern where I group

Jenkins build is back to normal : kafka-trunk-jdk8 #3253

2018-12-10 Thread Apache Jenkins Server
See

Build failed in Jenkins: kafka-trunk-jdk11 #150

2018-12-10 Thread Apache Jenkins Server
See Changes: [wangguoz] KAFKA-7610; Proactively timeout new group members if rebalance is -- [...truncated 2.24 MB...]

Re: [DISCUSS] KIP-307: Allow to define custom processor names with KStreams DSL

2018-12-10 Thread Guozhang Wang
I had one meta comment on the PR: https://github.com/apache/kafka/pull/5909#discussion_r240447153 On Mon, Dec 10, 2018 at 5:22 PM John Roesler wrote: > Hi Florian, > > I hope it's ok if I ask a few questions at this late stage... > > Comment 1 == > > It seems like the proposal is to add a

Re: [DISCUSS] KIP-307: Allow to define custom processor names with KStreams DSL

2018-12-10 Thread John Roesler
Hi Florian, I hope it's ok if I ask a few questions at this late stage... Comment 1 == It seems like the proposal is to add a new "Named" interface that is intended to be mixed in with the existing API objects at various points. Just to preface some of my comments, it looks like your KIP

[jira] [Resolved] (KAFKA-7610) Detect consumer failures in initial JoinGroup

2018-12-10 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-7610?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang resolved KAFKA-7610. -- Resolution: Fixed Assignee: Jason Gustafson (was: Boyang Chen) Fix Version/s:

Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

2018-12-10 Thread Adam Bellemare
1) I believe that the resolution mechanism John has proposed is sufficient - it is clean and easy and doesn't require additional RocksDB stores, which reduces the footprint greatly. I don't think we need to resolve based on timestamp or offset anymore, but if we decide to do to that would be

[jira] [Created] (KAFKA-7718) Allow customized header inheritance for stateful operators in DSL

2018-12-10 Thread Guozhang Wang (JIRA)
Guozhang Wang created KAFKA-7718: Summary: Allow customized header inheritance for stateful operators in DSL Key: KAFKA-7718 URL: https://issues.apache.org/jira/browse/KAFKA-7718 Project: Kafka

Re: [DISCUSS] KIP-382: MirrorMaker 2.0

2018-12-10 Thread Ryanne Dolan
Jun, thanks for your time reviewing the KIP. > In a MirrorSourceConnector, it seems that the offsets of the source will be stored in a different cluster from the target cluster? Jan Filipiak raised this issue as well, and suggested that no state be tracked in the source cluster. I've since

Re: [VOTE] KIP-394: Require member.id for initial join group request

2018-12-10 Thread Boyang Chen
Thanks a lot folks! I will start the implementation right away  Boyang From: Harsha Sent: Tuesday, December 11, 2018 3:28 AM To: dev@kafka.apache.org Subject: Re: [VOTE] KIP-394: Require member.id for initial join group request +1 . Thanks for the KIP. This is

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

2018-12-10 Thread Apache Jenkins Server
See

Re: [DISCUSS] KIP-392: Allow consumers to fetch from the closest replica

2018-12-10 Thread Jason Gustafson
Hey Mickael, Thanks for the comments. Responses below: - I'm guessing the selector will be invoke after each rebalance so > every time the consumer is assigned a partition it will be able to > select it. Is that true? I'm not sure it is necessary to do it after every rebalance, but certainly

[jira] [Resolved] (KAFKA-6036) Enable logical materialization to physical materialization

2018-12-10 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-6036?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang resolved KAFKA-6036. -- Resolution: Fixed Fix Version/s: 2.2.0 > Enable logical materialization to physical

[jira] [Created] (KAFKA-7717) Enable security configs in kafka.tools.EndToEndLatency

2018-12-10 Thread Stanislav Kozlovski (JIRA)
Stanislav Kozlovski created KAFKA-7717: -- Summary: Enable security configs in kafka.tools.EndToEndLatency Key: KAFKA-7717 URL: https://issues.apache.org/jira/browse/KAFKA-7717 Project: Kafka

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

2018-12-10 Thread Apache Jenkins Server
See Changes: [jason] KAFKA-7549; Old ProduceRequest with zstd compression does not return -- [...truncated 2.24 MB...] org.apache.kafka.streams.test.OutputVerifierTest >

Re: [DISCUSS] KIP-392: Allow consumers to fetch from the closest replica

2018-12-10 Thread Jason Gustafson
Hey Eno, Thanks for the comments. However, I'm a bit confused. I'm not suggesting we change Produce semantics in any way. All writes still go through the partition leader and nothing changes with respect to committing to the ISR. The main issue, as I've mentioned in the KIP, is the increased

Re: [VOTE] KIP-394: Require member.id for initial join group request

2018-12-10 Thread Harsha
+1 . Thanks for the KIP. This is very much needed. -Harsha On Mon, Dec 10, 2018, at 11:00 AM, Guozhang Wang wrote: > +1. Thanks Boyang! > > > Guozhang > > On Mon, Dec 10, 2018 at 10:29 AM Jason Gustafson wrote: > > > +1 Thanks for the KIP, Boyang! > > > > -Jason > > > > On Mon, Dec 10, 2018

Re: [Discuss] KIP-389: Enforce group.max.size to cap member metadata growth

2018-12-10 Thread Jason Gustafson
Hey Stanislav, Just to clarify, I think what you're suggesting is something like this in order to gracefully shrink the group: 1. Transition the group to PREPARING_REBALANCE. No members are kicked out. 2. Continue to allow offset commits and heartbeats for all current members. 3. Allow the first

Re: Kafka client Metadata update on demand?

2018-12-10 Thread Mayuresh Gharat
Hi Ming, Kafka clients do update there metadata on NotLeaderForPartitionException. The metadata update happens asynchronously. Also if you are getting this exception for a longer time, it might mean that your client is fetching metadata from a broker whose metadata cache is not updated with the

Re: [VOTE] KIP-394: Require member.id for initial join group request

2018-12-10 Thread Guozhang Wang
+1. Thanks Boyang! Guozhang On Mon, Dec 10, 2018 at 10:29 AM Jason Gustafson wrote: > +1 Thanks for the KIP, Boyang! > > -Jason > > On Mon, Dec 10, 2018 at 10:07 AM Boyang Chen wrote: > > > Thanks for voting my friends. Could someone give one more binding vote > > here? > > > > Best, > >

Finding reviewers for some fixes for sequenceId overflow

2018-12-10 Thread Ming Liu
Hi, community, Here is the simple fix for two serious problems about sequenceId overflow (one in client, one at broker). It has been there for a week. Can somebody review it? https://issues.apache.org/jira/browse/KAFKA-7693 https://issues.apache.org/jira/browse/KAFKA-7692 Thanks! Ming

Kafka client Metadata update on demand?

2018-12-10 Thread Ming Liu
Hey community, It seems Kafka Metadata update only happens at the pre-configured Metadata update interval. During upgrade, when leader changes, the client will fail with NotLeaderForPartitionException until next Metadata update happened. I wonder why we don't have the on-demand Metadata

Re: [VOTE] KIP-394: Require member.id for initial join group request

2018-12-10 Thread Jason Gustafson
+1 Thanks for the KIP, Boyang! -Jason On Mon, Dec 10, 2018 at 10:07 AM Boyang Chen wrote: > Thanks for voting my friends. Could someone give one more binding vote > here? > > Best, > Boyang > > From: Bill Bejeck > Sent: Thursday, December 6, 2018 2:45 AM > To:

Re: [VOTE] KIP-345: Introduce static membership protocol to reduce consumer rebalances

2018-12-10 Thread Boyang Chen
Hey there, could I get more votes on this thread? Thanks for the vote from Mayuresh and Mike :) Best, Boyang From: Mayuresh Gharat Sent: Thursday, December 6, 2018 10:53 AM To: dev@kafka.apache.org Subject: Re: [VOTE] KIP-345: Introduce static membership

Re: [VOTE] KIP-394: Require member.id for initial join group request

2018-12-10 Thread Boyang Chen
Thanks for voting my friends. Could someone give one more binding vote here? Best, Boyang From: Bill Bejeck Sent: Thursday, December 6, 2018 2:45 AM To: dev@kafka.apache.org Subject: Re: [VOTE] KIP-394: Require member.id for initial join group request +1 Thanks

[jira] [Resolved] (KAFKA-7549) Old ProduceRequest with zstd compression does not return error to client

2018-12-10 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-7549?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson resolved KAFKA-7549. Resolution: Fixed > Old ProduceRequest with zstd compression does not return error to

Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

2018-12-10 Thread John Roesler
Hi, all, >> In fact, we >> can just keep a single final-result store with timestamps and reject values >> that have a smaller timestamp, is that right? > Which is the correct output should at least be decided on the offset of > the original message. Thanks for this point, Jan. KIP-258 is

Re: [DISCUSS] KIP-398: Support reading trust store from classpath

2018-12-10 Thread Noa Resare
Thank you for your comments, see replies inline. > On 9 Dec 2018, at 01:33, Harsha wrote: > > Hi Noa, > Based on KIP"s motivation section > "If we had the ability to load a trust store from the classpath as well as > from a file, the trust store could be shipped in a jar that could

Re: [DISCUSS] KIP-391: Allow Producing with Offsets for Cluster Replication

2018-12-10 Thread Edoardo Comar
(shameless bump) any additional feedback is welcome ... thanks! Edoardo Comar wrote on 27/11/2018 15:35:09: > From: Edoardo Comar > To: dev@kafka.apache.org > Date: 27/11/2018 15:35 > Subject: Re: [DISCUSS] KIP-391: Allow Producing with Offsets for > Cluster Replication > > Hi Jason > > we

[jira] [Created] (KAFKA-7716) Unprocessed messages when Broker fails

2018-12-10 Thread Finbarr Naughton (JIRA)
Finbarr Naughton created KAFKA-7716: --- Summary: Unprocessed messages when Broker fails Key: KAFKA-7716 URL: https://issues.apache.org/jira/browse/KAFKA-7716 Project: Kafka Issue Type: Bug

Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

2018-12-10 Thread Jan Filipiak
On 10.12.2018 07:42, Guozhang Wang wrote: > Hello Adam / Jan / John, > > Sorry for being late on this thread! I've finally got some time this > weekend to cleanup a load of tasks on my queue (actually I've also realized > there are a bunch of other things I need to enqueue while cleaning them up