Re: [DISCUSS] KIP-219 - Improve Quota Communication

2017-11-03 Thread Becket Qin
Thanks Rajini. 1. Good point. We do need to bump up the protocol version so that the new clients do not wait for another throttle time when they are talking to old brokers. I'll update the KIP. 2. That is true. But the client was not supposed to send request to the broker during that period anywa

Re: [DISCUSS] KIP-220: Add AdminClient into Kafka Streams' ClientSupplier

2017-11-03 Thread Ted Yu
Looks good overall. bq. the creation within StreamsPartitionAssignor Typo above: should be StreamPartitionAssignor On Fri, Nov 3, 2017 at 4:49 PM, Guozhang Wang wrote: > Hello folks, > > I have filed a new KIP on adding AdminClient into Streams for internal > topic management. > > Looking for

[GitHub] kafka pull request #4176: KAFKA-6168 Connect Schema comparison is slow for l...

2017-11-03 Thread tedyu
GitHub user tedyu opened a pull request: https://github.com/apache/kafka/pull/4176 KAFKA-6168 Connect Schema comparison is slow for large schemas Re-arrange order of comparisons in equals() to evaluate non-composite fields first Cache hash code You can merge this pull request i

Re: [VOTE] KIP-215: Add topic regex support for Connect sinks

2017-11-03 Thread Stephane Maarek
+1! The S3 or hdfs connector will now be super powerful ! On 4 Nov. 2017 11:27 am, "Konstantine Karantasis" wrote: > Nice addition! > > +1 (non-binding) > > Konstantine > > On Fri, Nov 3, 2017 at 4:52 PM, Jeff Klukas wrote: > > > So sorry for skirting the process there. I wasn't aware of the 72

Re: [DISCUSS] KIP-220: Add AdminClient into Kafka Streams' ClientSupplier

2017-11-03 Thread Matt Farmer
This seems like an A+ improvement to me. On Fri, Nov 3, 2017 at 7:49 PM Guozhang Wang wrote: > Hello folks, > > I have filed a new KIP on adding AdminClient into Streams for internal > topic management. > > Looking for feedback on > > * > https://cwiki.apache.org/confluence/display/KAFKA/KIP-220

Re: [VOTE] KIP-215: Add topic regex support for Connect sinks

2017-11-03 Thread Konstantine Karantasis
Nice addition! +1 (non-binding) Konstantine On Fri, Nov 3, 2017 at 4:52 PM, Jeff Klukas wrote: > So sorry for skirting the process there. I wasn't aware of the 72 hour > window and I don't see that mentioned in in > https://cwiki.apache.org/confluence/display/KAFKA/Bylaws#Bylaws-Voting > > Sho

Re: [VOTE] KIP-215: Add topic regex support for Connect sinks

2017-11-03 Thread Jeff Klukas
So sorry for skirting the process there. I wasn't aware of the 72 hour window and I don't see that mentioned in in https://cwiki.apache.org/confluence/display/KAFKA/Bylaws#Bylaws-Voting Should I feel free to update that wiki page with a note about the window? On Fri, Nov 3, 2017 at 7:49 PM, Ewen

[DISCUSS] KIP-220: Add AdminClient into Kafka Streams' ClientSupplier

2017-11-03 Thread Guozhang Wang
Hello folks, I have filed a new KIP on adding AdminClient into Streams for internal topic management. Looking for feedback on *https://cwiki.apache.org/confluence/display/KAFKA/KIP-220%3A+Add+AdminClient+into+Kafka+Streams%27+ClientSupplier

Re: [VOTE] KIP-215: Add topic regex support for Connect sinks

2017-11-03 Thread Ewen Cheslack-Postava
Jeff, Just FYI re: process, I think you're pretty much definitely in the clear hear since this one is a straightforward design I doubt anybody would object to, but voting normally stays open 72h to ensure everyone has a chance to weigh in. Again thanks for the KIP and we can move any final discus

Re: [VOTE] KIP-215: Add topic regex support for Connect sinks

2017-11-03 Thread Jeff Klukas
Looks like we've achieved lazy majority, so I'll move the KIP to approved. Thanks all for looking this over. On Fri, Nov 3, 2017 at 7:31 PM, Jason Gustafson wrote: > +1. Thanks for the KIP! > > On Fri, Nov 3, 2017 at 2:15 PM, Guozhang Wang wrote: > > > +1 binding > > > > On Fri, Nov 3, 2017 at

[jira] [Created] (KAFKA-6170) Add the AdminClient in Streams' KafkaClientSupplier

2017-11-03 Thread Guozhang Wang (JIRA)
Guozhang Wang created KAFKA-6170: Summary: Add the AdminClient in Streams' KafkaClientSupplier Key: KAFKA-6170 URL: https://issues.apache.org/jira/browse/KAFKA-6170 Project: Kafka Issue Type:

Re: [VOTE] KIP-215: Add topic regex support for Connect sinks

2017-11-03 Thread Jason Gustafson
+1. Thanks for the KIP! On Fri, Nov 3, 2017 at 2:15 PM, Guozhang Wang wrote: > +1 binding > > On Fri, Nov 3, 2017 at 1:25 PM, Ewen Cheslack-Postava > wrote: > > > +1 binding > > > > Thanks Jeff! > > > > On Wed, Nov 1, 2017 at 5:21 PM, Randall Hauch wrote: > > > > > +1 (non-binding) > > > > > >

Re: [DISCUSS]KIP-216: IQ should throw different exceptions for different errors

2017-11-03 Thread Guozhang Wang
Thanks for writing up the KIP. Vito, Matthias: one thing that I wanted to figure out first is what categories of errors we want to notify the users, if we only wants to distinguish fatal v.s. retriable then probably we should rename the proposed StateStoreMigratedException / StateStoreClosedExcept

Re: [DISCUSS] KIP-219 - Improve Quota Communication

2017-11-03 Thread Rajini Sivaram
Hi Becket, Thank you for the KIP. A few comments: 1.KIP says: "*No public interface changes are needed. We only propose behavior change on the broker side.*" But from the proposed changes, it sounds like clients will be updated to wait for throttle-time before sending next response, and also no

[jira] [Created] (KAFKA-6169) Kafka Log should reject negative timestamps

2017-11-03 Thread Ryan P (JIRA)
Ryan P created KAFKA-6169: - Summary: Kafka Log should reject negative timestamps Key: KAFKA-6169 URL: https://issues.apache.org/jira/browse/KAFKA-6169 Project: Kafka Issue Type: Bug Repo

[jira] [Created] (KAFKA-6168) Connect Schema comparison is slow for large schemas

2017-11-03 Thread Randall Hauch (JIRA)
Randall Hauch created KAFKA-6168: Summary: Connect Schema comparison is slow for large schemas Key: KAFKA-6168 URL: https://issues.apache.org/jira/browse/KAFKA-6168 Project: Kafka Issue Type:

Re: [VOTE] KIP-215: Add topic regex support for Connect sinks

2017-11-03 Thread Guozhang Wang
+1 binding On Fri, Nov 3, 2017 at 1:25 PM, Ewen Cheslack-Postava wrote: > +1 binding > > Thanks Jeff! > > On Wed, Nov 1, 2017 at 5:21 PM, Randall Hauch wrote: > > > +1 (non-binding) > > > > Thanks for pushing this through. Great work! > > > > Randall Hauch > > > > On Wed, Nov 1, 2017 at 9:40 AM

Re: [VOTE] KIP-215: Add topic regex support for Connect sinks

2017-11-03 Thread Ewen Cheslack-Postava
+1 binding Thanks Jeff! On Wed, Nov 1, 2017 at 5:21 PM, Randall Hauch wrote: > +1 (non-binding) > > Thanks for pushing this through. Great work! > > Randall Hauch > > On Wed, Nov 1, 2017 at 9:40 AM, Jeff Klukas wrote: > > > I haven't heard any additional concerns over the proposal, so I'd like

Re: [DISCUSS] KIP-219 - Improve Quota Communication

2017-11-03 Thread Becket Qin
Thanks for the comment, Jun, 1. Yes, you are right. This could also happen with the current quota mechanism because we are essentially muting the socket during throttle time. There might be two ways to solve this. A) use another socket to send heartbeat. B) let the GroupCoordinator know that the c

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

2017-11-03 Thread Apache Jenkins Server
See

Build failed in Jenkins: kafka-trunk-jdk7 #2946

2017-11-03 Thread Apache Jenkins Server
See Changes: [wangguoz] HOTFIX: Add missing template ref in upgrade section -- [...truncated 382.92 KB...] kafka.api.TransactionsTest > testFencingOnSendOffsets STARTED k

Build failed in Jenkins: kafka-trunk-jdk9 #169

2017-11-03 Thread Apache Jenkins Server
See Changes: [wangguoz] HOTFIX: Add missing template ref in upgrade section -- Started by an SCM change [EnvInject] - Loading node environment variables. Building remotely

Re: [DISCUSS] KIP-219 - Improve Quota Communication

2017-11-03 Thread Jun Rao
Hi, Jiangjie, Thanks for bringing this up. A couple of quick thoughts. 1. If the throttle time is large, what can happen is that a consumer won't be able to heart beat to the group coordinator frequent enough. In that case, even with this KIP, it seems there could be frequent consumer group rebal

Re: [VOTE] KIP-203: Add toLowerCase support to sasl.kerberos.principal.to.local rule

2017-11-03 Thread Manikumar
Bump up. waiting for few more binding votes. On Wed, Oct 18, 2017 at 6:57 PM, Rajini Sivaram wrote: > +1 (binding) > > On Mon, Oct 9, 2017 at 5:32 PM, Manikumar > wrote: > > > I'm bumping this up to get some attention :) > > > > On Wed, Sep 27, 2017 at 8:46 PM, Tom Bentley > > wrote: > > > > >

Build failed in Jenkins: kafka-trunk-jdk9 #168

2017-11-03 Thread Apache Jenkins Server
See Changes: [wangguoz] MINOR: Update docs for new version -- Started by an SCM change [EnvInject] - Loading node environment variables. Building remotely on ubuntu-4 (ubun

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

2017-11-03 Thread Apache Jenkins Server
See Changes: [wangguoz] MINOR: Update docs for new version -- [...truncated 380.72 KB...] kafka.server.epoch.LeaderEpochFileCacheTest > shouldNotResetEpochHistoryTailIfU

[jira] [Created] (KAFKA-6167) Timestamp on streams directory contains a colon, which is an illegal character

2017-11-03 Thread Justin Manchester (JIRA)
Justin Manchester created KAFKA-6167: Summary: Timestamp on streams directory contains a colon, which is an illegal character Key: KAFKA-6167 URL: https://issues.apache.org/jira/browse/KAFKA-6167

[GitHub] kafka pull request #4169: MINOR: Update docs for new version

2017-11-03 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/4169 ---

[GitHub] kafka pull request #4175: KAFKA-6161 add base classes for (De)Serializers wi...

2017-11-03 Thread evis
GitHub user evis opened a pull request: https://github.com/apache/kafka/pull/4175 KAFKA-6161 add base classes for (De)Serializers with empty conf methods All (de)serializers, which have empty configure() and/or close() methods, are now inherit NoConf(De)Serializer. Also, such classe

[jira] [Resolved] (KAFKA-6159) Link to upgrade docs in 1.0.0 release notes is broken

2017-11-03 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-6159?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang resolved KAFKA-6159. -- Resolution: Fixed > Link to upgrade docs in 1.0.0 release notes is broken >

[GitHub] kafka pull request #4174: MINOR: Add pull request template

2017-11-03 Thread ijuma
GitHub user ijuma opened a pull request: https://github.com/apache/kafka/pull/4174 MINOR: Add pull request template You can merge this pull request into a Git repository by running: $ git pull https://github.com/ijuma/kafka pull-request-template Alternatively you can review a

Jenkins build is back to normal : kafka-trunk-jdk7 #2944

2017-11-03 Thread Apache Jenkins Server
See

[jira] [Created] (KAFKA-6166) Streams configuration requires consumer. and producer. in order to be read

2017-11-03 Thread Justin Manchester (JIRA)
Justin Manchester created KAFKA-6166: Summary: Streams configuration requires consumer. and producer. in order to be read Key: KAFKA-6166 URL: https://issues.apache.org/jira/browse/KAFKA-6166 Proj

Build failed in Jenkins: kafka-trunk-jdk7 #2943

2017-11-03 Thread Apache Jenkins Server
See Changes: [rajinisivaram] KAFKA-6060; Add workload generation capabilities to Trogdor -- [...truncated 382.47 KB...] kafka.security.auth.ZkAuthorizationTest > testZkUt

Build failed in Jenkins: kafka-trunk-jdk9 #167

2017-11-03 Thread Apache Jenkins Server
See Changes: [rajinisivaram] KAFKA-6060; Add workload generation capabilities to Trogdor -- Started by an SCM change [EnvInject] - Loading node environment variables. Build

[jira] [Resolved] (KAFKA-6060) Add workload generation capabilities to Trogdor

2017-11-03 Thread Rajini Sivaram (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-6060?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Rajini Sivaram resolved KAFKA-6060. --- Resolution: Fixed Fix Version/s: 1.1.0 Issue resolved by pull request 4073 https://gith

[GitHub] kafka pull request #4073: KAFKA-6060: Add workload generation capabilities t...

2017-11-03 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/4073 ---