Re: [DISCUSS] KIP-447: Producer scalability for exactly once semantics

2019-06-24 Thread Boyang Chen
Hey Jason, thank you for the proposal here. Some of my thoughts below. On Mon, Jun 24, 2019 at 8:58 PM Jason Gustafson wrote: > Hi Boyang, > > Thanks for picking this up! Still reading through the updates, but here are > a couple initial comments on the APIs: > > 1. The `TxnProducerIdentity`

Re: [DISCUSS] KIP-447: Producer scalability for exactly once semantics

2019-06-24 Thread Jason Gustafson
Hi Boyang, Thanks for picking this up! Still reading through the updates, but here are a couple initial comments on the APIs: 1. The `TxnProducerIdentity` class is a bit awkward. I think we are trying to encapsulate state from the current group assignment. Maybe something like

Re: [DISCUSS] KIP-478 Strongly Typed Processor API

2019-06-24 Thread Guozhang Wang
Hi John, Yeah I think we should not do all the repackaging as part of this KIP as well (we can just do the movement of the Processor / ProcessorSupplier), but I think we need to discuss the end goal here since otherwise we may do the repackaging of Processor in this KIP, but only later on

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

2019-06-24 Thread Apache Jenkins Server
See

Re: [DISCUSS] KIP-480 : Sticky Partitioner

2019-06-24 Thread Boyang Chen
Thank you Justine for the KIP! Do you mind creating a corresponding JIRA ticket too? On Mon, Jun 24, 2019 at 4:51 PM Colin McCabe wrote: > Hi Justine, > > Thanks for the KIP. This looks great! > > In one place in the KIP, you write: "Remove > testRoundRobinWithUnavailablePartitions() and

[DISCUSS] KIP-481: SerDe Improvements for Connect Decimal type in JSON

2019-06-24 Thread Almog Gavra
Hi Everyone! Kicking off discussion for a new KIP: https://cwiki.apache.org/confluence/display/KAFKA/KIP-481%3A+SerDe+Improvements+for+Connect+Decimal+type+in+JSON For those who are interested, I have a prototype implementation that helped guide my design: https://github.com/agavra/kafka/pull/1

Re: [DISCUSS] KIP-480 : Sticky Partitioner

2019-06-24 Thread Colin McCabe
Hi Justine, Thanks for the KIP. This looks great! In one place in the KIP, you write: "Remove testRoundRobinWithUnavailablePartitions() and testRoundRobin() since the round robin functionality of the partitioner has been removed." You can skip this and similar lines. We don't need to

[DISCUSS] KIP-390: Allow fine-grained configuration for compression (Rebooted)

2019-06-24 Thread Dongjin Lee
Hello. Here is the new discussion thread for KIP-390: Allow fine-grained configuration for compression. https://cwiki.apache.org/confluence/display/KAFKA/KIP-390%3A+Allow+fine-grained+configuration+for+compression Here is some history: Initially, the draft implementation was done with

[jira] [Created] (KAFKA-8595) Support SerDe of Decimals in JSON that are not HEX encoded

2019-06-24 Thread Almog Gavra (JIRA)
Almog Gavra created KAFKA-8595: -- Summary: Support SerDe of Decimals in JSON that are not HEX encoded Key: KAFKA-8595 URL: https://issues.apache.org/jira/browse/KAFKA-8595 Project: Kafka Issue

Re: Preliminary blog post for the Apache Kafka 2.3.0 release

2019-06-24 Thread Colin McCabe
Hi Stephane, Sounds interesting! Do you have a link to the video you made for 2.3? best, Colin On Sun, Jun 23, 2019, at 15:10, Stephane Maarek wrote: > The video is ready :) waiting for the release of Kafka 2.3 to make it > public. @colin if you want to link it in the blog at some point

[jira] [Created] (KAFKA-8594) Add Kafka Streams compatibility test for Kafka 2.3.0

2019-06-24 Thread Colin P. McCabe (JIRA)
Colin P. McCabe created KAFKA-8594: -- Summary: Add Kafka Streams compatibility test for Kafka 2.3.0 Key: KAFKA-8594 URL: https://issues.apache.org/jira/browse/KAFKA-8594 Project: Kafka Issue

Re: [DISCUSS] KIP-478 Strongly Typed Processor API

2019-06-24 Thread John Roesler
Hey Guozhang, Thanks for the idea! I'm wondering if we could take a middle ground and take your proposed layout as a "roadmap", while only actually moving the classes that are already involved in this KIP. The reason I ask is not just to control the scope of this KIP, but also, I think that if

[DISCUSS] KIP-480 : Sticky Partitioner

2019-06-24 Thread Justine Olshan
Hello, This is the discussion thread for KIP-480: Sticky Partitioner. https://cwiki.apache.org/confluence/display/KAFKA/KIP-480%3A+Sticky+Partitioner Thank you, Justine Olshan

Re: Permission to create KIP

2019-06-24 Thread Jun Rao
Hi, Justine, Thanks for your interest. Just gave you the wiki permission. Jun On Mon, Jun 24, 2019 at 12:19 PM Justine Olshan wrote: > Hi, I was wondering if I could have permission to create a KIP. My wiki > username is jolshan. > > Thank you, > Justine Olshan >

Permission to create KIP

2019-06-24 Thread Justine Olshan
Hi, I was wondering if I could have permission to create a KIP. My wiki username is jolshan. Thank you, Justine Olshan

Re: [DISCUSS] KIP-479: Add Materialized to Join

2019-06-24 Thread John Roesler
Thanks Guozhang, Yep. Maybe we can consider just exactly what the join needs: > the WindowStore itself is fine, if overly broad, > since the only two methods we need are `window.put(key, value, > context().timestamp())` and `WindowStoreIterator iter = > window.fetch(key, timeFrom, timeTo)`. One

Re: [DISCUSS] KIP-479: Add Materialized to Join

2019-06-24 Thread Guozhang Wang
Hello John, My main concern is exactly the first point at the bottom of your analysis here: "* configure the bytes store". I'm not sure if using a window bytes store would be ideal for stream-stream windowed join; e.g. we could consider two dimensional list sorted by timestamps and then by keys

[RESULT] [VOTE] 2.3.0 RC3

2019-06-24 Thread Colin McCabe
Hi all, This vote passes with 6 +1 votes (3 of which are binding) and no 0 or -1 votes. Thanks to everyone who voted. +1 votes PMC Members: * Ismael Juma * Guozhang Wang * Gwen Shapira Community: * Kamal Chandraprakash * Jakub Scholz * Mickael Maison 0 votes * No votes -1 votes * No votes

Re: [DISCUSS] KIP-447: Producer scalability for exactly once semantics

2019-06-24 Thread Boyang Chen
Thank you Ismael for the suggestion. We will attempt to address it by giving more details to rejected alternative section. Thank you for the comment Guozhang! Answers are inline below. On Sun, Jun 23, 2019 at 6:33 PM Guozhang Wang wrote: > Hello Boyang, > > Thanks for the KIP, I have some

Re: Preliminary blog post for the Apache Kafka 2.3.0 release

2019-06-24 Thread Colin McCabe
Hi Ismael, Good idea. I added a section on this. best, Colin On Fri, Jun 21, 2019, at 17:30, Ismael Juma wrote: > Thanks Colin! Maybe we should mention that restarts are much faster when > you have a lot of partitions: > https://issues.apache.org/jira/browse/KAFKA-7283 > > "We did

Re: [DISCUSS] KIP-479: Add Materialized to Join

2019-06-24 Thread John Roesler
Hey Guozhang and Bill, For what it's worth, I agree with you both! I think it might help the discussion to look concretely at what Materialized does: * set a WindowBytesStoreSupplier * set a name * set the key/value serdes * disable/enable/configure change-logging * disable/enable caching *

[jira] [Created] (KAFKA-8593) Provide a JSON Array Serde for serializing a List to a JSON Array and vice versa

2019-06-24 Thread Oliver Weiler (JIRA)
Oliver Weiler created KAFKA-8593: Summary: Provide a JSON Array Serde for serializing a List to a JSON Array and vice versa Key: KAFKA-8593 URL: https://issues.apache.org/jira/browse/KAFKA-8593

[jira] [Created] (KAFKA-8592) Broker Dynamic Configuration fails to resolve variables as per KIP-421

2019-06-24 Thread TEJAL ADSUL (JIRA)
TEJAL ADSUL created KAFKA-8592: -- Summary: Broker Dynamic Configuration fails to resolve variables as per KIP-421 Key: KAFKA-8592 URL: https://issues.apache.org/jira/browse/KAFKA-8592 Project: Kafka

Re: [VOTE] KIP-476: Add Java AdminClient interface

2019-06-24 Thread Andy Coates
Hi all, KIP updated: - No deprecation - Factory method back onto Admin interface I'd like to kick off another round of voting please. Thanks, Andy On Mon, 24 Jun 2019 at 16:03, Andy Coates wrote: > I agree Matthias. > > (In Scala, such factory methods are on a companion object. As Java

Re: [VOTE] KIP-476: Add Java AdminClient interface

2019-06-24 Thread Andy Coates
I agree Matthias. (In Scala, such factory methods are on a companion object. As Java doesn't have the concept of a companion object, an equivalent would be a utility class with a similar name...) However, I'll update the KIP to include the factory method on the interface. On Fri, 21 Jun 2019 at

Re: [VOTE] KIP-476: Add Java AdminClient interface

2019-06-24 Thread Andy Coates
That makes a lot of sense. OK, no deprecation. On Fri, 21 Jun 2019 at 15:11, Ismael Juma wrote: > This is even more reason not to deprecate immediately, there is very little > maintenance cost for us. We should be mindful that many of our users (eg > Spark, Flink, etc.) typically allow users

Re: [DISCUSS] KIP-470: TopologyTestDriver test input and output usability improvements

2019-06-24 Thread Bill Bejeck
Hi Jukka > These topic objects are only interfacing TopologyTestDriver, not affecting > the internal functionality of it. In my plan the internal data structures > are using those Producer/ConsumerRecords as earlier. That way I don't see > how those could be affected. I mistakenly thought the

[jira] [Created] (KAFKA-8591) NPE when reloading connector configuration using WorkerConfigTransformer

2019-06-24 Thread Nacho Munoz (JIRA)
Nacho Munoz created KAFKA-8591: -- Summary: NPE when reloading connector configuration using WorkerConfigTransformer Key: KAFKA-8591 URL: https://issues.apache.org/jira/browse/KAFKA-8591 Project: Kafka

Re: [VOTE] 2.3.0 RC3

2019-06-24 Thread Mickael Maison
Thanks Colin for making a new RC for KAFKA-8564. +1 (non binding) I checked signatures and ran quickstart on the 2.12 binary On Mon, Jun 24, 2019 at 6:03 AM Gwen Shapira wrote: > > +1 (binding) > Verified signatures, verified good build on jenkins, built from > sources anyway and ran quickstart