Re: KAFKA-8584: Support of ByteBuffer for bytes field implemented[Convert Kafka RPCs to use automatically generated code]

2019-09-19 Thread Nikolay Izhikov
Hello, guys. Looks like we have duplicate tickets and PR's here. One from me: KAFKA-8584: Support of ByteBuffer for bytes field implemented. ticket - https://issues.apache.org/jira/browse/KAFKA-8584 pr - https://github.com/apache/kafka/pull/7342 and one from Colin McCabe: KAFKA-8628: Auto-gene

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

2019-09-19 Thread Apache Jenkins Server
See Changes: [github] MINOR: Update dependencies for Kafka 2.4 (part 2) (#7333) -- [...truncated 2.63 MB...] org.apache.kafka.streams.test.OutputVerifierTest > shouldFai

Re: [VOTE] KIP-511: Collect and Expose Client's Name and Version in the Brokers

2019-09-19 Thread David Jacot
Hi all, I have updated the KIP to incorporate Colin's feedback. Best, David On Thu, Sep 19, 2019 at 8:44 AM David Jacot wrote: > Hi Colin, > > Thank you for your feedback! Please find my comments/answers below: > > *> Nitpick: in the intro paragraph, "Operators of Apache Kafka clusters > have

[jira] [Created] (KAFKA-8924) Default grace period (-1) of TimeWindows causes suppress to never emit events

2019-09-19 Thread Jira
Michał created KAFKA-8924: - Summary: Default grace period (-1) of TimeWindows causes suppress to never emit events Key: KAFKA-8924 URL: https://issues.apache.org/jira/browse/KAFKA-8924 Project: Kafka

RE: [DISCUSS] KIP-519: Make SSL context/engine configuration extensible

2019-09-19 Thread Pellerin, Clement
I appreciate the effort you put into this. Lets do this in steps. You had a question on getConfiguredInstance(). The method getConfiguredInstance(key, Class) implemented in AbstractConfig is how the MetricsReporter and other extension points are intantiated. Creating the extension point this wa

Re: [DISCUSS] KIP-525 - Return topic metadata and configs in CreateTopics response

2019-09-19 Thread Rajini Sivaram
Hi Colin, Thanks for reviewing the KIP! I have added default values for the RPC. Since other int fields seem to be using -1, I used -1 as the default for NumPartitions and ReplicationFactor. I think a single method that returns TopicConfig seems better because tools that are interested in the re

Re: [Discuss] - KIP-532 - Add KStream#toTable to the Streams DSL

2019-09-19 Thread aishwarya kumar
Thanks Matthias, That does make sense, let me update the KIP to reflect the Materialization scenario. Best, Aishwarya On Tue, Sep 17, 2019, 2:49 PM Matthias J. Sax wrote: > Aishwarya, > > thanks for the KIP. Overall, I think it makes sense to allow converting > a KStream into a KTable. > > Fro

Re: [DISCUSS] KIP-515: Reorganize checkpoint system in log cleaner to be per partition

2019-09-19 Thread Jason Gustafson
Hi Richard, Just reposting my comment from the JIRA: The underlying problem here also impacts the cleaning of transaction markers. We use the same delete horizon in order to tell when it is safe to remove the marker. If all the data from a transaction has been cleaned and the delete horizon has p

[VOTE] KIP-521: Redirect Connect log4j messages also to a file by default

2019-09-19 Thread Konstantine Karantasis
I'd like to start the vote on KIP-521. The proposal seems straightforward and no major concerns came up during its recent brief discussions. I'd appreciate your votes and, of course, your comments are still welcome. Hopefully we could meet the forthcoming KIP deadline for this simple yet useful o

[jira] [Created] (KAFKA-8925) Flaky Test kafka.api.AuthorizerIntegrationTest.testIdempotentProducerNoIdempotentWriteAclInProduce

2019-09-19 Thread Guozhang Wang (Jira)
Guozhang Wang created KAFKA-8925: Summary: Flaky Test kafka.api.AuthorizerIntegrationTest.testIdempotentProducerNoIdempotentWriteAclInProduce Key: KAFKA-8925 URL: https://issues.apache.org/jira/browse/KAFKA-8925

Re: [DISCUSS] KIP-525 - Return topic metadata and configs in CreateTopics response

2019-09-19 Thread Colin McCabe
On Thu, Sep 19, 2019, at 06:31, Rajini Sivaram wrote: > Hi Colin, > > Thanks for reviewing the KIP! > > I have added default values for the RPC. Since other int fields seem to be > using -1, I used -1 as the default for NumPartitions and ReplicationFactor. > Thanks. > I think a single method t

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

2019-09-19 Thread John Roesler
I'm +1 either way :) -John On Wed, Sep 18, 2019 at 5:37 PM Bill Bejeck wrote: > > Good catch! I meant to propose the name to be "StreamJoin". I have updated > the KIP accordingly. > > As for the name, I originally had "StreamJoined" and updated it after some > comments on the KIP. > I do feel th

Re: [VOTE] KIP-521: Redirect Connect log4j messages also to a file by default

2019-09-19 Thread Gwen Shapira
+1 (binding) On Thu, Sep 19, 2019 at 8:53 AM Konstantine Karantasis wrote: > > I'd like to start the vote on KIP-521. > > The proposal seems straightforward and no major concerns came up during its > recent brief discussions. I'd appreciate your votes and, of course, your > comments are still wel

[DISCUSS] KIP-527: Add NothingSerde to Serdes

2019-09-19 Thread Nikolay Izhikov
All, I'd like to start a discussion for adding a NothingSerde to Serdes. https://cwiki.apache.org/confluence/display/KAFKA/KIP-527%3A+Add+NothingSerde+to+Serdes Your comments and suggestions are welcome. signature.asc Description: This is a digitally signed message part

Re: [DISCUSS] KIP-527: Add NothingSerde to Serdes

2019-09-19 Thread Andrew Otto
Why 'NothingSerdes' instead of 'NullSerdes'? On Thu, Sep 19, 2019 at 1:10 PM Nikolay Izhikov wrote: > All, > > I'd like to start a discussion for adding a NothingSerde to Serdes. > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-527%3A+Add+NothingSerde+to+Serdes > > Your comments and s

Re: [DISCUSS] KIP-527: Add NothingSerde to Serdes

2019-09-19 Thread Nikolay Izhikov
Hello, Andrew. Seems, usage null or nothing is matter of taste. I dont mind if we call it NullSerde чт, 19 сент. 2019 г., 20:28 Andrew Otto : > Why 'NothingSerdes' instead of 'NullSerdes'? > > On Thu, Sep 19, 2019 at 1:10 PM Nikolay Izhikov > wrote: > > > All, > > > > I'd like to start a discus

Re: [DISCUSS] KIP-525 - Return topic metadata and configs in CreateTopics response

2019-09-19 Thread Rajini Sivaram
Thanks, Colin! That makes sense. I have updated the KIP to use separate methods. Can you do take a quick look to see if the KIP is ready for voting? Thank you, Rajini On Thu, Sep 19, 2019 at 5:22 PM Colin McCabe wrote: > On Thu, Sep 19, 2019, at 06:31, Rajini Sivaram wrote: > > Hi Colin, > > >

Re: [DISCUSS] KIP-527: Add NothingSerde to Serdes

2019-09-19 Thread Andrew Otto
NullSerdes seems more descriptive, but up to you! :) On Thu, Sep 19, 2019 at 1:37 PM Nikolay Izhikov wrote: > Hello, Andrew. > > Seems, usage null or nothing is matter of taste. I dont mind if we call it > NullSerde > > чт, 19 сент. 2019 г., 20:28 Andrew Otto : > > > Why 'NothingSerdes' instead

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

2019-09-19 Thread Matthias J. Sax
As I mentioned on the DISCUSS thread, it think either `StreamsJoined` (plural) or `StreamJoin` are good names. But I am also ok with `StreamJoined` if anyone insist on it. I leave it up to Bill to pick any of the three variant. +1 (binding) -Matthias On 9/19/19 9:40 AM, John Roesler wrote: > I'

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

2019-09-19 Thread Bill Bejeck
Thanks for the comments, Matthias. I don't have a strong preference, so given that Matthias is ok with "StreamJoined" and Guozhang seems to prefer "StreamJoined" I'll update the KIP accordingly. Thanks, Bill On Thu, Sep 19, 2019 at 11:04 AM Matthias J. Sax wrote: > As I mentioned on the DISC

Re: [VOTE] KIP-517: Add consumer metrics to observe user poll behavior

2019-09-19 Thread Harsha Chintalapani
+1 (binding). Thanks for the KIP. -Harsha On Wed, Sep 18, 2019 at 9:07 AM Mickael Maison wrote: > +1 (non binding) > Thanks for the KIP, I can see this being really useful! > > On Wed, Sep 18, 2019 at 4:40 PM Kevin Lu wrote: > > > > Hi All, > > > > Since we have a bit of support, I'd like to s

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

2019-09-19 Thread Almog Gavra
Thanks everyone for the voting - the PR is now ready for review! https://github.com/apache/kafka/pull/7354 Cheers, Almog On Tue, Sep 17, 2019 at 12:56 PM Guozhang Wang wrote: > +1 (binding). > > Thanks Almog! > > Guozhang > > On Tue, Sep 17, 2019 at 11:56 AM Bill Bejeck wrote: > > > +1 (bindin

[jira] [Created] (KAFKA-8926) Log Cleaner thread dies when log.cleaner.min.cleanable.ratio is set to 0

2019-09-19 Thread Jordan Mcmillan (Jira)
Jordan Mcmillan created KAFKA-8926: -- Summary: Log Cleaner thread dies when log.cleaner.min.cleanable.ratio is set to 0 Key: KAFKA-8926 URL: https://issues.apache.org/jira/browse/KAFKA-8926 Project: K

Re: [VOTE] KIP-511: Collect and Expose Client's Name and Version in the Brokers

2019-09-19 Thread Colin McCabe
On Wed, Sep 18, 2019, at 23:44, David Jacot wrote: > Hi Colin, > > Thank you for your feedback! Please find my comments/answers below: > > *> Nitpick: in the intro paragraph, "Operators of Apache Kafka clusters > have literally no information about the clients connected to their > clusters" seems

Re: [DISCUSS] KIP-525 - Return topic metadata and configs in CreateTopics response

2019-09-19 Thread Colin McCabe
+1 for starting the vote. Thanks, Rajini! C. On Thu, Sep 19, 2019, at 10:47, Rajini Sivaram wrote: > Thanks, Colin! That makes sense. I have updated the KIP to use separate > methods. Can you do take a quick look to see if the KIP is ready for voting? > > Thank you, > > Rajini > > On Thu, Sep

Re: [VOTE] KIP-496: Administrative API to delete consumer offsets

2019-09-19 Thread Colin McCabe
Sounds good to me. It makes sense to add this functionality to the command line. best, Colin On Wed, Sep 18, 2019, at 11:26, David Jacot wrote: > Indeed, I have forgotten to add the action. There will be a new action « > —delete-offsets ». Sorry! > > *Proposed API* > kafka-consumer-groups.sh

Re: [DISCUSS] KIP-525 - Return topic metadata and configs in CreateTopics response

2019-09-19 Thread Rajini Sivaram
Thanks Colin! On Thu, Sep 19, 2019 at 9:54 PM Colin McCabe wrote: > +1 for starting the vote. Thanks, Rajini! > > C. > > On Thu, Sep 19, 2019, at 10:47, Rajini Sivaram wrote: > > Thanks, Colin! That makes sense. I have updated the KIP to use separate > > methods. Can you do take a quick look to

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

2019-09-19 Thread Guozhang Wang
Thanks Bill for the update, I'm +1 as well (binding). On Thu, Sep 19, 2019 at 11:25 AM Bill Bejeck wrote: > Thanks for the comments, Matthias. > > I don't have a strong preference, so given that Matthias is ok with > "StreamJoined" and Guozhang seems to prefer "StreamJoined" I'll update the > KI

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

2019-09-19 Thread Apache Jenkins Server
See

Re: [DISCUSS] KIP-515: Reorganize checkpoint system in log cleaner to be per partition

2019-09-19 Thread Richard Yu
Hi Jason, That hadn't occurred to me. I think I missed your comment in the discussion, so I created this KIP only with resolving the problem regarding tombstones. Whats your thoughts? If the problem regarding transaction markers is a little too complex, then we can we just leave it out of the KIP

[jira] [Created] (KAFKA-8927) Remove config `partition.grouper` and interface `PartitionGrouper`

2019-09-19 Thread Matthias J. Sax (Jira)
Matthias J. Sax created KAFKA-8927: -- Summary: Remove config `partition.grouper` and interface `PartitionGrouper` Key: KAFKA-8927 URL: https://issues.apache.org/jira/browse/KAFKA-8927 Project: Kafka

Re: [DISCUSS] KIP-507: Securing Internal Connect REST Endpoints

2019-09-19 Thread Chris Egerton
Hi all, Two quick updates on KIP-507: 1) According to https://docs.oracle.com/javase/8/docs/technotes/guides/security/crypto/CryptoSpec.html#Mac, "With some MAC algorithms, the (secret-)key algorithm associated with the (secret-)key object used to initialize the Mac object does not matter (this i

[VOTE] KIP-528: Deprecate PartitionGrouper configuration and interface

2019-09-19 Thread Matthias J. Sax
Hi, I would like to propose a small KIP to deprecate some public APIs that are considered "dangerous" to use, with the goal to remove them in the next major release. Because the KIP is straight forward and to make it into 2.4 release, I call directly for a vote. https://cwiki.apache.org/confluen

Re: [VOTE] KIP-528: Deprecate PartitionGrouper configuration and interface

2019-09-19 Thread Guozhang Wang
+1, we should do that long ago :) Guozhang On Thu, Sep 19, 2019 at 4:28 PM Matthias J. Sax wrote: > Hi, > > I would like to propose a small KIP to deprecate some public APIs that > are considered "dangerous" to use, with the goal to remove them in the > next major release. > > Because the KIP i

Re: [VOTE] KIP-528: Deprecate PartitionGrouper configuration and interface

2019-09-19 Thread Bill Bejeck
Thanks for the KIP Matthias. I agree with Guozhang on this one. +1 (binding) -Bill On Thu, Sep 19, 2019 at 4:41 PM Guozhang Wang wrote: > +1, we should do that long ago :) > > Guozhang > > On Thu, Sep 19, 2019 at 4:28 PM Matthias J. Sax > wrote: > > > Hi, > > > > I would like to propose a sm

Re: [DISCUSS] KIP-519: Make SSL context/engine configuration extensible

2019-09-19 Thread Maulin Vasavada
Hi Clement, Thanks for pointing to AbstractConfig. Now I understand what you were saying. I'll respond by tonight with more thoughts. Thanks Maulin On Thu, Sep 19, 2019 at 5:46 AM Pellerin, Clement wrote: > I appreciate the effort you put into this. > > Lets do this in steps. You had a questio

[VOTE] KIP-507: Securing Internal Connect REST Endpoints

2019-09-19 Thread Chris Egerton
Hi all, I'd like to begin voting on KIP-507: https://cwiki.apache.org/confluence/display/KAFKA/KIP-507%3A+Securing+Internal+Connect+REST+Endpoints Thanks to Ryanne, Magesh, Konstantine, Greg, and Randall for the fruitful discussion! Cheers, Chris

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

2019-09-19 Thread Matthias J. Sax
Personally, I think it would be cleaner to just pass `Deserializer` into `InputTopic` and `Serializer` into `OutputTopic`. Keeping the interface simple is a good argument and we should not have overloads -- but it seems that picking `Serdes` instead of the more straight forward `(De)Serializer` is

Re: [DISCUSS] KIP-213: Second follow-up on Foreign Key Joins

2019-09-19 Thread Matthias J. Sax
Sorry that I never replied. I am fine with the current proposal :) -Matthias On 7/12/19 5:09 AM, Adam Bellemare wrote: > @Matthias J. Sax - Thoughts on the > semantics of simply leaving it as-is, with the extra tombstones? As John > put it: "It may be unnecessary to

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

2019-09-19 Thread John Roesler
Hey, all, For what it's worth, I agree with Matthias. While it's true that you're likely to have Serdes on hand, if for whatever reason, you don't, it's a pain to make one. Plus, with just requiring Serializer and Deserializer, each component is only asking for the minimum thing it needs, rather

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

2019-09-19 Thread John Roesler
FWIW, I'm also +1 (non-binding). Thanks for tackling this, Bill. -John On Thu, Sep 19, 2019 at 3:09 PM Guozhang Wang wrote: > > Thanks Bill for the update, I'm +1 as well (binding). > > On Thu, Sep 19, 2019 at 11:25 AM Bill Bejeck wrote: > > > Thanks for the comments, Matthias. > > > > I don't

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

2019-09-19 Thread Apache Jenkins Server
See

Re: [DISCUSS] KIP-519: Make SSL context/engine configuration extensible

2019-09-19 Thread Maulin Vasavada
Hi Clement So assuming there are two classes - SslFactory and SslEngineFactory like I suggested in my detailed post before this, we can use config.getConfiguredInstance() in SslFactory for SslEngineFactory class configuration and then followed by init() method. I don't see a challenge there. Can y

Re: [DISCUSS] Apache Kafka 2.4.0 release

2019-09-19 Thread Paul Whalen
Manikumar, KIP-401 was accepted a few weeks ago and there is a PR pending review, can it be included in the release as well? Thanks, Paul On Mon, Sep 16, 2019 at 6:14 AM Manikumar wrote: > Hi All, > > Just a reminder that any new/pending KIP must pass vote by next Wednesday > (Sep 25, 2019) to

[jira] [Resolved] (KAFKA-8915) Unable to modify partition

2019-09-19 Thread huxihx (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-8915?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] huxihx resolved KAFKA-8915. --- Resolution: Not A Problem > Unable to modify partition > -- > > Key:

Re: [ DISCUSS ] KIP-512:Adding headers to RecordMetaData

2019-09-19 Thread Maulin Vasavada
Hi all, Can this KIP get to voting? I don't see any major concerns so far. It is already adding ProducerRecord in Producer Interceptor which was one of the main ask. Thanks Maulin On Mon, Sep 16, 2019 at 4:09 PM Renuka M wrote: > Hi All, > > The motivation behind this KIP is have info about re

is reduce partition necessary

2019-09-19 Thread Xiangyuan LI
hi: so far kafka doesn't support reduce partition num for some reason, but sometimes we need to have this function to resolve some problems. 1. we have some topics with too many partitions(more than 1000) used 2 years, and lots of consumer groups subscribe them, now these topics impact system

RE: [DISCUSS] KIP-519: Make SSL context/engine configuration extensible

2019-09-19 Thread Pellerin, Clement
We will get there eventually but I need to address another point first. My goal is to do exactly what the "other extension points with reconfigurable custom configs" are doing unless there is a good reason not to. They provide a ready-made solution that will let us reuse code, avoid pitfalls and

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

2019-09-19 Thread Apache Jenkins Server
See Changes: [github] MINOR: Add last modified time and deletion horizon to clear log message [matthias] [KAFKA-7994] Improve Stream time accuracy for restarts and rebalances [jason] MINOR: Send latest Leader

Re: [DISCUSS] Apache Kafka 2.4.0 release

2019-09-19 Thread Manikumar
Hi, Added KIP-401 to the wiki page for tracking. Thanks, On Fri, Sep 20, 2019 at 7:55 AM Paul Whalen wrote: > Manikumar, > > KIP-401 was accepted a few weeks ago and there is a PR pending review, can > it be included in the release as well? > > Thanks, > Paul > > On Mon, Sep 16, 2019 at 6:14

Re: [VOTE] KIP-517: Add consumer metrics to observe user poll behavior

2019-09-19 Thread Manikumar
+1 (binding), Thanks for the KIP. On Fri, Sep 20, 2019 at 12:41 AM Harsha Chintalapani wrote: > +1 (binding). Thanks for the KIP. > > -Harsha > > On Wed, Sep 18, 2019 at 9:07 AM Mickael Maison > wrote: > > > +1 (non binding) > > Thanks for the KIP, I can see this being really useful! > > > > On

Re: [DISCUSS] KIP-527: Add NothingSerde to Serdes

2019-09-19 Thread Nikolay Izhikov
Hello, Andrew. OK, if nobody mind, let's change it to Null. В Чт, 19/09/2019 в 13:54 -0400, Andrew Otto пишет: > NullSerdes seems more descriptive, but up to you! :) > > On Thu, Sep 19, 2019 at 1:37 PM Nikolay Izhikov wrote: > > > Hello, Andrew. > > > > Seems, usage null or nothing is matter

Re: [DISCUSS] KIP-519: Make SSL context/engine configuration extensible

2019-09-19 Thread Maulin Vasavada
Hi Clement There will be good amount of state in the SslEngineFactory's default implementation. Hence I feel we might anyway have a chaperon class to provide reconfigurable functionality and will have one more class to host the state/behavior of actual SSLContext/SSLEngine creation. While doing th

Re: [VOTE] KIP-517: Add consumer metrics to observe user poll behavior

2019-09-19 Thread Maulin Vasavada
+1 (non-binding). Thanks for the KIP. On Thu, Sep 19, 2019 at 10:38 PM Manikumar wrote: > +1 (binding), Thanks for the KIP. > > On Fri, Sep 20, 2019 at 12:41 AM Harsha Chintalapani > wrote: > > > +1 (binding). Thanks for the KIP. > > > > -Harsha > > > > On Wed, Sep 18, 2019 at 9:07 AM Mickael M

Re: [VOTE] KIP-517: Add consumer metrics to observe user poll behavior

2019-09-19 Thread Tom Bentley
+1 (non-binding). On Fri, Sep 20, 2019 at 7:00 AM Maulin Vasavada wrote: > +1 (non-binding). Thanks for the KIP. > > On Thu, Sep 19, 2019 at 10:38 PM Manikumar > wrote: > > > +1 (binding), Thanks for the KIP. > > > > On Fri, Sep 20, 2019 at 12:41 AM Harsha Chintalapani > > wrote: > > > > > +1

Re: [DISCUSS] KIP-519: Make SSL context/engine configuration extensible

2019-09-19 Thread Maulin Vasavada
Overall my thinking is - When somebody wants to customize creation of SSLEngine, most likely they are more expert in dealing with SSL domain related stuff than "Kafka's reconfigurability" aspect. As a custom implementation it makes more sense to me to say - Hey I'll control how I initialize my SSL