[DISCUSS] KIP-344: The auto-generated client id should be passed to MetricsReporter

2018-07-20 Thread Kevin Lu
Hi All, This is the discussion thread for KIP-344 ( https://cwiki.apache.org/confluence/display/KAFKA/KIP-344%3A+The+auto-generated+client+id+should+be+passed+to+MetricsReporter ). Thanks! Regards, Kevin

Re: [ANNOUNCE] Apache Kafka 1.1.1 Released

2018-07-20 Thread Dongjin Lee
Thank you for great work! :) On Fri, Jul 20, 2018 at 11:25 AM Guozhang Wang wrote: > Dong, thanks for driving the release! > > On Thu, Jul 19, 2018 at 6:51 PM, Ismael Juma wrote: > > > Thank you for managing the release Dong! > > > > Ismael > > > > On Thu, 19 Jul 2018, 16:54 Dong Lin, wrote: >

Re: Contribution request

2018-07-20 Thread András Katona
thank you On Thu, Jul 19, 2018 at 6:14 PM Matthias J. Sax wrote: > Done > > On 7/19/18 8:49 AM, András Katona wrote: > > Hello Team, > > > > Could you add me to the contributors on Apache Kafka project in JIRA, > > please? > > Username: akatona > > > > Thank you, > > Andras Katona > > > >

Re: [DISCUSS] KIP-344: The auto-generated client id should be passed to MetricsReporter

2018-07-20 Thread Ted Yu
Looks good to me. On Fri, Jul 20, 2018 at 12:17 AM Kevin Lu wrote: > Hi All, > > This is the discussion thread for KIP-344 ( > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-344%3A+The+auto-generated+client+id+should+be+passed+to+MetricsReporter > ). > > Thanks! > > Regards, > Kevin >

[jira] [Created] (KAFKA-7187) offsetsForTimes in MockConsumer implementation

2018-07-20 Thread Jing Chen (JIRA)
Jing Chen created KAFKA-7187: Summary: offsetsForTimes in MockConsumer implementation Key: KAFKA-7187 URL: https://issues.apache.org/jira/browse/KAFKA-7187 Project: Kafka Issue Type: Improvement

Re: [VOTE] KIP-331 Add default implementation to close() and configure() for Serializer, Deserializer and Serde

2018-07-20 Thread Chia-Ping Tsai
> The KIP needs 3 binding votes to pass. Thanks for the reminder. I will reopen the ballot box until we get 3 tickets. > I still think we should include the details of how things will look like > with the headers being passed to serializers/deserializers to ensure > things actually make sense as

Re: [VOTE] KIP-331 Add default implementation to close() and configure() for Serializer, Deserializer and Serde

2018-07-20 Thread Ismael Juma
Part of the motivation for this KIP is to make these interfaces functional interfaces. But I think that may not be desirable due to the method that passes headers. So, it doesn't make sense to discuss two separate changes to the same interfaces in isolation, we should figure out how we want them to

Requesting review of MINOR PR, make regular expressions statically compiled to prevent constant recompilation.

2018-07-20 Thread Koen De Groote
Hi all, The PR: https://github.com/apache/kafka/pull/5168 This MINOR PR has been around for a while, guess I'll poke here a bit. Poking was suggested somewhere, I recall... What does the PR do? It makes statically compiled regular expressions to replace the current in-line regular expressions, w

Re: [VOTE] KIP-331 Add default implementation to close() and configure() for Serializer, Deserializer and Serde

2018-07-20 Thread Chia-Ping Tsai
> Part of the motivation for this KIP is to make these interfaces functional > interfaces. But I think that may not be desirable due to the method that > passes headers. So, it doesn't make sense to discuss two separate changes > to the same interfaces in isolation, we should figure out how we want

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

2018-07-20 Thread Apache Jenkins Server
See Changes: [lindong28] KAFKA-7098; Improve accuracy of throttling by avoiding under-estimating -- [...truncated 2.63 MB...] org.apache.kafka.connect.runtime.ConnectMet

Build failed in Jenkins: kafka-trunk-jdk10 #298

2018-07-20 Thread Apache Jenkins Server
See Changes: [github] MINOR: Fix transient test failure in SslTransportLayerTest (#5396) [github] KAFKA-3702: Change log level of SSL close_notify failure (#5397) -- [...

Re: [DISCUSSION] KIP-336: Consolidate ExtendedSerializer/Serializer and ExtendedDeserializer/Deserializer

2018-07-20 Thread Chia-Ping Tsai
> The approach where all the methods have a default implementation and the > user chooses to override one of them seems the most appealing to me given > the current state. sounds good. one Q: Which implementation is suitable for serialize()/deserialize()? maybe just throw exception? BTW, making

Build failed in Jenkins: kafka-2.0-jdk8 #78

2018-07-20 Thread Apache Jenkins Server
See Changes: [rajinisivaram] MINOR: Fix transient test failure in SslTransportLayerTest (#5396) [rajinisivaram] KAFKA-3702: Change log level of SSL close_notify failure (#5397)

[jira] [Created] (KAFKA-7188) Avoid reverse DNS lookup in SASL channel builder

2018-07-20 Thread Rajini Sivaram (JIRA)
Rajini Sivaram created KAFKA-7188: - Summary: Avoid reverse DNS lookup in SASL channel builder Key: KAFKA-7188 URL: https://issues.apache.org/jira/browse/KAFKA-7188 Project: Kafka Issue Type:

[jira] [Created] (KAFKA-7189) Add a Merge Transformer for Kafka Connect

2018-07-20 Thread Abdelhamide EL ARIB (JIRA)
Abdelhamide EL ARIB created KAFKA-7189: -- Summary: Add a Merge Transformer for Kafka Connect Key: KAFKA-7189 URL: https://issues.apache.org/jira/browse/KAFKA-7189 Project: Kafka Issue Typ

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

2018-07-20 Thread Apache Jenkins Server
See Changes: [github] MINOR: Fix transient test failure in SslTransportLayerTest (#5396) [github] KAFKA-3702: Change log level of SSL close_notify failure (#5397) -- [...

Re: [DISCUSS] KIP-342 Add Customizable SASL extensions to OAuthBearer authentication

2018-07-20 Thread Ron Dagostino
Hi Stanislav. I'm wondering if we should make SaslExtensions part of the public API. I mentioned this in my review of the PR, too (and tagged Rajini to get her input). If we add a Map to the Subject's public credentials we are basically making a public commitment that any Map associated with the

[jira] [Resolved] (KAFKA-7081) Add describe all topics API to AdminClient

2018-07-20 Thread Manikumar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-7081?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar resolved KAFKA-7081. -- Resolution: Won't Fix Closing this KIP in favor of adding filtering support to the Metadata API   http

Re: [DISCUSS] KIP-289: Improve the default group id behavior in KafkaConsumer

2018-07-20 Thread Jason Gustafson
Hi Vahid, Sorry for getting to this so late. I think there are two things here: 1. The use of "" as a groupId has always been a dubious practice at best. We definitely ought to deprecate its use in the client. Perhaps in the next major release, we can remove support completely. However, since old

Re: [DISCUSS] KIP-342 Add Customizable SASL extensions to OAuthBearer authentication

2018-07-20 Thread Stanislav Kozlovski
Hi Ron, I saw that and decided that would be the best approach. The current ScramExtensions implementation uses a Map in the public credentials and I thought I would follow convention rather than introduce my own thing, but maybe this is best On Fri, Jul 20, 2018 at 8:39 AM Ron Dagostino wrote:

Re: [VOTE] KIP-322: Return new error code for DeleteTopics API when topic deletion disabled.

2018-07-20 Thread Manikumar
Bumping this thread. On Tue, Jul 10, 2018 at 10:33 PM Manikumar wrote: > Waiting for one more binding vote to pass this minor KIP. Appreciate your > vote. > > On Wed, Jul 4, 2018 at 7:03 PM Eno Thereska > wrote: > >> +1 (non binding) >> >> On Wed, Jul 4, 2018 at 1:19 PM, Kamal Chandraprakash <

Re: [Discuss] KIP-321: Add method to get TopicNameExtractor in TopologyDescription

2018-07-20 Thread Matthias J. Sax
I would still deprecate existing `topics()` method. If users need a String, they can call `topicSet().toString()`. It's just a personal preference, because I believe it's good to keep the API "minimal". About renaming the other methods: I thinks it's a very small burden to deprecate the existing

Re: [DISCUSS] KIP-289: Improve the default group id behavior in KafkaConsumer

2018-07-20 Thread Rajini Sivaram
+1 to deprecate use of "" as group.id since it is odd to have a resource name that you cannot set ACLs for. Agree, we have to support older clients though. Thanks, Rajini On Fri, Jul 20, 2018 at 5:25 PM, Jason Gustafson wrote: > Hi Vahid, > > Sorry for getting to this so late. I think there ar

Re: [VOTE] KIP-322: Return new error code for DeleteTopics API when topic deletion disabled.

2018-07-20 Thread Jason Gustafson
Hey Manikumar, This looks good. Do we have to keep the current behavior when handling old DeleteTopics versions? I'm wondering if it would be better to return an UNKNOWN_ERROR (say) and let the client raise that to the user instead of just timing out. Thanks, Jason On Fri, Jul 20, 2018 at 10:00

Re: [VOTE] KIP-322: Return new error code for DeleteTopics API when topic deletion disabled.

2018-07-20 Thread Manikumar
Hi Jason, Yes, we can return UNKNOWN_SERVER_ERROR for older versions. This will give better developer experience. Updated the KIP. Thanks, On Fri, Jul 20, 2018 at 11:02 PM Jason Gustafson wrote: > Hey Manikumar, > > This looks good. Do we have to keep the current behavior when handling old >

Re: [DISCUSS] KIP-289: Improve the default group id behavior in KafkaConsumer

2018-07-20 Thread Dhruvil Shah
If we are looking into deprecating the empty group id, would it also make sense to have the same character restriction for it as that for topic names? We have stricter validation for topic names but none for group id and transaction id. I think we should (eventually) make character restriction the

Re: [DISCUSS] KIP-289: Improve the default group id behavior in KafkaConsumer

2018-07-20 Thread Stanislav Kozlovski
I agree with Jason's notion that > implicit use of the empty group.id to commit offsets is more likely to be causing users unexpected problems than actually providing a useful capability. I was initially confused that this is the behavior when investigating a new-ish JIRA issue

Re: [DISCUSS]: KIP-339: Create a new ModifyConfigs API

2018-07-20 Thread Colin McCabe
I updated the KIP. https://cwiki.apache.org/confluence/display/KAFKA/KIP-339%3A+Create+a+new+IncrementalAlterConfigs+API Updates: * Use "incrementalAlterConfigs" rather than "modifyConfigs," for consistency with the other "alter" APIs. * Implement Magnus' idea of supporting "append" and "subtra

[jira] [Created] (KAFKA-7190) Under low traffic conditions purging repartition topics cause WARN statements about UNKNOWN_PRODUCER_ID

2018-07-20 Thread Bill Bejeck (JIRA)
Bill Bejeck created KAFKA-7190: -- Summary: Under low traffic conditions purging repartition topics cause WARN statements about UNKNOWN_PRODUCER_ID Key: KAFKA-7190 URL: https://issues.apache.org/jira/browse/KAFKA-719

Jenkins build is back to normal : kafka-trunk-jdk10 #299

2018-07-20 Thread Apache Jenkins Server
See

Re: [VOTE] KIP-322: Return new error code for DeleteTopics API when topic deletion disabled.

2018-07-20 Thread Jason Gustafson
After thinking about it, perhaps a better option is INVALID_REQUEST? I think this is usually used for requests which are malformed in some way, so it's a slight abuse, but at least it suggests an incompatibility between what the client is requesting to do and what the server supports. UNKNOWN_SERVE

Re: [VOTE] KIP-322: Return new error code for DeleteTopics API when topic deletion disabled.

2018-07-20 Thread Manikumar
Thanks for the feedback, sounds good to me. Updated the KIP. On Sat, Jul 21, 2018 at 12:04 AM Jason Gustafson wrote: > After thinking about it, perhaps a better option is INVALID_REQUEST? I > think this is usually used for requests which are malformed in some way, so > it's a slight abuse, but a

Re: [VOTE] KIP-322: Return new error code for DeleteTopics API when topic deletion disabled.

2018-07-20 Thread Jason Gustafson
Thanks, +1 from me. On Fri, Jul 20, 2018 at 11:41 AM, Manikumar wrote: > Thanks for the feedback, sounds good to me. Updated the KIP. > > On Sat, Jul 21, 2018 at 12:04 AM Jason Gustafson > wrote: > > > After thinking about it, perhaps a better option is INVALID_REQUEST? I > > think this is usua

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

2018-07-20 Thread Apache Jenkins Server
See Changes: [wangguoz] MINOR: Fix format violations streams scala tests (#5402) [jason] KAFKA-7141; Consumer group describe should include groups with no -- [...truncate

Build failed in Jenkins: kafka-1.1-jdk7 #166

2018-07-20 Thread Apache Jenkins Server
See Changes: [jason] MINOR: Add thread dumps if broker node cannot be stopped (#5373) -- [...truncated 397.82 KB...] at java.io.FileOutputStream.writeBytes(Native Met

[jira] [Resolved] (KAFKA-7185) getMatchingAcls throws StringIndexOutOfBoundsException for empty resource name

2018-07-20 Thread Rajini Sivaram (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-7185?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Rajini Sivaram resolved KAFKA-7185. --- Resolution: Fixed Reviewer: Ismael Juma Fix Version/s: 2.0.0 > getMatchingAcl

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

2018-07-20 Thread Apache Jenkins Server
See Changes: [github] MINOR: fix upgrade docs for Streams (#5394) -- [...truncated 435.49 KB...] kafka.server.epoch.LeaderEpochFileCacheTest > shouldAddEpochAndMessageOffse

Re: [DISCUSS] KIP-320: Allow fetchers to detect and handle log truncation

2018-07-20 Thread Guozhang Wang
Hi Jason, The proposed API seems reasonable to me too. Could you please also update the wiki page ( https://cwiki.apache.org/confluence/display/KAFKA/KIP-320%3A+Allow+fetchers+to+detect+and+handle+log+truncation) with a section say "workflow" on how the proposed API will be co-used with others to:

Discussion: New components in JIRA?

2018-07-20 Thread Ray Chiang
I've been doing a little bit of component cleanup in JIRA.  What do people think of adding one or more of the following components? - logging: For any consumer/producer/broker logging (i.e. log4j). This should help disambiguate from the "log" component (i.e. Kafka messages). - mirrormaker: Th

Build failed in Jenkins: kafka-trunk-jdk10 #301

2018-07-20 Thread Apache Jenkins Server
See Changes: [rajinisivaram] KAFKA-7185: Allow empty resource name when matching ACLs (#5400) -- [...truncated 1.54 MB...] kafka.security.auth.ZkAuthorizationTest > testZ

Build failed in Jenkins: kafka-2.0-jdk8 #80

2018-07-20 Thread Apache Jenkins Server
See Changes: [rajinisivaram] KAFKA-7185: Allow empty resource name when matching ACLs (#5400) -- [...truncated 2.48 MB...] org.apache.kafka.streams.TopologyTest > shouldNotA

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

2018-07-20 Thread Apache Jenkins Server
See Changes: [github] KAFKA-5037 Follow-up: move Scala test to Java (#5399) [matthias] MINOR: fix upgrade docs for Streams (#5394) [jason] MINOR: Add thread dumps if broker node cannot be stopped (#5373)

Re: [DISCUSS] KIP-334 Include partitions in exceptions raised during consumer record deserialization/validation

2018-07-20 Thread Stanislav Kozlovski
Hey group, I added a Pull Request for this KIP - here it is https://github.com/apache/kafka/pull/5410 Please take a look. Best, Stanislav On Thu, Jul 5, 2018 at 11:06 AM Ismael Juma wrote: > Yes, the Scala consumers have been removed in 2.0.0, which simplifies some > of this. The following com

Re: [DISCUSS] KIP-334 Include partitions in exceptions raised during consumer record deserialization/validation

2018-07-20 Thread Ted Yu
Hi, Stanislav: Do you plan to start VOTE thread ? Cheers On Fri, Jul 20, 2018 at 6:11 PM Stanislav Kozlovski wrote: > Hey group, > > I added a Pull Request for this KIP - here it is > https://github.com/apache/kafka/pull/5410 > Please take a look. > > Best, > Stanislav > > On Thu, Jul 5, 2018 a

Re: [DISCUSS] KIP-334 Include partitions in exceptions raised during consumer record deserialization/validation

2018-07-20 Thread Stanislav Kozlovski
Hi Ted, I do plan to start one. When is the appropriate time? My reasoning was that people would like to view the changes first On Fri, Jul 20, 2018, 6:21 PM Ted Yu wrote: > Hi, Stanislav: > Do you plan to start VOTE thread ? > > Cheers > > On Fri, Jul 20, 2018 at 6:11 PM Stanislav Kozlovski <

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

2018-07-20 Thread Apache Jenkins Server
See Changes: [rajinisivaram] KAFKA-7185: Allow empty resource name when matching ACLs (#5400) -- [...truncated 429.66 KB...] kafka.utils.SchedulerTest > testMockScheduler

Re: [Discuss] KIP-321: Add method to get TopicNameExtractor in TopologyDescription

2018-07-20 Thread Nishanth Pradeep
I definitely agree with you on deprecating topics(). I also think changing the method names for consistency is reasonable, since there is no functionality change. Although, I can be convinced either way on this one. Best, Nishanth Pradeep On Fri, Jul 20, 2018 at 12:15 PM Matthias J. Sax wrote:

Jenkins build is back to normal : kafka-1.1-jdk7 #167

2018-07-20 Thread Apache Jenkins Server
See

Jenkins build is back to normal : kafka-trunk-jdk10 #302

2018-07-20 Thread Apache Jenkins Server
See

Re: [VOTE] KIP-340: Allow kafka-reassign-partitions.sh and kafka-log-dirs.sh to take admin client property file

2018-07-20 Thread Joel Koshy
+1 On Tue, Jul 17, 2018 at 1:42 PM, Ted Yu wrote: > +1 > > On Tue, Jul 17, 2018 at 1:40 PM Jason Gustafson > wrote: > > > +1. This is useful (though the naming inconsistencies in the tools are > > vexing, as always). > > > > -Jason > > > > On Tue, Jul 17, 2018 at 12:24 PM, Dong Lin wrote: > >

Re: [VOTE] KIP-340: Allow kafka-reassign-partitions.sh and kafka-log-dirs.sh to take admin client property file

2018-07-20 Thread Dong Lin
Thanks to everyone who voted and provided feedback! This KIP is now adopted with 3 binding +1s (Jason, Joel, Dong) and 1 non-binding +1 (Ted). Regards, Dong On Fri, Jul 20, 2018 at 11:20 PM, Joel Koshy wrote: > +1 > > On Tue, Jul 17, 2018 at 1:42 PM, Ted Yu wrote: > > > +1 > > > > On Tue, Jul

Re: [VOTE] KIP-322: Return new error code for DeleteTopics API when topic deletion disabled.

2018-07-20 Thread Manikumar
*Hi All,The vote has passed with 3 binding votes (Gwen, Harsha, Jason) and 8 non-binding votes (Vahid, Mickael, Ted, Daniele, Satish, Magnus, Kamal, Eno).Thanks everyone for the votes.Thanks,Manikumar* On Sat, Jul 21, 2018 at 12:26 AM Jason Gustafson wrote: > Thanks, +1 from me. > > On Fri, Jul

Re: [DISCUSS] KIP-263: Allow broker to skip sanity check of inactive segments on broker startup

2018-07-20 Thread Dong Lin
Here is some information related to the KIP. Previously we thought we can ignore the sanity check by default to speedup broker startup and there is no need for extra configuration. However based on the code and some previous experiment result it is not clear whether it will work. When LogManager.c