Re: [DISCUSS] KIP-290: Support for wildcard suffixed ACLs

2018-05-21 Thread Ismael Juma
It got 3 binding votes already and the deadline is only tomorrow. :) Ismael On Mon, 21 May 2018, 21:50 Colin McCabe, wrote: > On Mon, May 21, 2018, at 04:53, Andy Coates wrote: > > Hey Piyush, > > > > Thanks for the updated KIP! Couple of minor points from me: > > > > When

Re: [DISCUSS] KIP-290: Support for wildcard suffixed ACLs

2018-05-21 Thread Colin McCabe
On Mon, May 21, 2018, at 04:53, Andy Coates wrote: > Hey Piyush, > > Thanks for the updated KIP! Couple of minor points from me: > > When storing wildcard-suffixed Acls in ZK, drop the asterisk of the end for > the path, e.g. change "*/kafka-wildcard-acl/Topic/teamA*" * to "*/* >

Re: [DISCUSS] KIP-290: Support for wildcard suffixed ACLs

2018-05-21 Thread Colin McCabe
Oops, will post on the vote thread. best, Colin On Mon, May 21, 2018, at 21:15, Colin McCabe wrote: > On Mon, May 21, 2018, at 04:53, Andy Coates wrote: > > Hey Piyush, > > > > Thanks for the updated KIP! Couple of minor points from me: > > > > When storing wildcard-suffixed Acls in ZK, drop

Re: [VOTE] KIP-290: Support for wildcard suffixed ACLs

2018-05-21 Thread Colin McCabe
Hmm, do we still need one more binding +1, or did I misread the vote thread? +1 (non-binding) from me. As I posted in the other thread, I think it would make sense to name the new ZK hierarchy /kafka-prefix-acls or similar, to reflect the fact that they are ACLs that match by a name prefix.

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

2018-05-21 Thread Apache Jenkins Server
See Changes: [github] MINOR: Ignore test_broker_type_bounce_at_start system test (#5055) -- [...truncated 423.43 KB...] kafka.admin.ResetConsumerGroupOffsetTest >

Re: [DISCUSS] KIP-305: Add Connect primitive number converters

2018-05-21 Thread Ewen Cheslack-Postava
Sorry, probably not ideal to just be seeing this now given KIP deadline, but in general straightforward ones where we expect things to be uncontroversial, I think its fine to kick off a vote thread. Worst case we have people re-file votes if something substantial changes in the proposal. -Ewen

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

2018-05-21 Thread Apache Jenkins Server
See Changes: [github] MINOR: Ignore test_broker_type_bounce_at_start system test (#5055) -- [...truncated 1.06 MB...]

Re: [VOTE] KIP-306: Configuration for Delaying Response to Failed Client Authentication

2018-05-21 Thread Jason Gustafson
+1. Just one nit: could we use an INT type for the config? I can't imagine that not being enough. -Jason On Mon, May 21, 2018 at 3:59 PM, Ismael Juma wrote: > Thanks for the KIP, +1 (binding). > > Ismael > > On Mon, May 21, 2018 at 7:52 AM Dhruvil Shah

Re: [DISCUSS] KIP-304: Connect runtime mode improvements for container platforms

2018-05-21 Thread Ewen Cheslack-Postava
Hey all, I think think this is a great discussion, and is helping to clarify the real pain points as well as explore a few more options than just what was initially proposed. Stephane, I think why you're ending up in "grand redesign" state is because you're highlighting (and the KIP's motivation

Re: [VOTE] KIP-303: Add Dynamic Routing Support in Kafka Streams' Topology Sink

2018-05-21 Thread Gwen Shapira
Recasting my +1 On Mon, May 21, 2018 at 3:38 PM, Guozhang Wang wrote: > Thanks Matthias, will do. > > On Mon, May 21, 2018 at 3:35 PM, Matthias J. Sax > wrote: > > > Thanks. I am fine with changing the `StreamPartitioner` interface > directly. > > > >

Re: KAFKA-6733 feedback

2018-05-21 Thread Ismael Juma
Hi Mateusz, This is a good change, but it requires a KIP as mentioned in the PR. You gave an example of a similar PR without a KIP, but there was a KIP for it: https://cwiki.apache.org/confluence/display/KAFKA/KIP-265%3A+Make+Windowed+Serde+to+public+APIs Ismael On Mon, May 21, 2018 at 4:00 PM

KAFKA-6733 feedback

2018-05-21 Thread Mateusz Zakarczemny
Hi, Could I ask for some feedback regarding https://github.com/apache/kafka/pull/4807 ? It's waiting 1,5 month. I had to resolve conflicts with trunk couple of times. I would be grateful if someone could take a look it. Regards, Mateusz Zakarczemny

Re: [VOTE] KIP-306: Configuration for Delaying Response to Failed Client Authentication

2018-05-21 Thread Ismael Juma
Thanks for the KIP, +1 (binding). Ismael On Mon, May 21, 2018 at 7:52 AM Dhruvil Shah wrote: > Hi, > > I would like to start a vote on KIP-306 which proposes to add a > configuration to delay responses to failed authentication. > > Link to the KIP: > >

RE: Kafka consumer to unzip stream of .gz files and read

2018-05-21 Thread Koushik Chitta
You should read the message value as byte array rather than string . Other Approach is , while producing you can use the kafka compression = GZIP to have similar results. -Original Message- From: mayur shah Sent: Monday, May 21, 2018 1:50 AM To:

Re: [VOTE] KIP-306: Configuration for Delaying Response to Failed Client Authentication

2018-05-21 Thread Edoardo Comar
+1 (non-binding) Thanks ! On 21 May 2018 at 19:18, Bill Bejeck wrote: > Thanks for the KIP. > > +1 > > -Bill > > On Mon, May 21, 2018 at 1:41 PM, Ted Yu wrote: > >> +1 >> >> On Mon, May 21, 2018 at 9:02 AM, Guozhang Wang wrote: >> >>

Re: [VOTE] KIP-303: Add Dynamic Routing Support in Kafka Streams' Topology Sink

2018-05-21 Thread Guozhang Wang
Thanks Matthias, will do. On Mon, May 21, 2018 at 3:35 PM, Matthias J. Sax wrote: > Thanks. I am fine with changing the `StreamPartitioner` interface directly. > > Can you add the idea bout `DynamicStreamPartitioner` to "rejected > alternative" section. Thx. > > >

Re: [VOTE] KIP-303: Add Dynamic Routing Support in Kafka Streams' Topology Sink

2018-05-21 Thread Matthias J. Sax
Thanks. I am fine with changing the `StreamPartitioner` interface directly. Can you add the idea bout `DynamicStreamPartitioner` to "rejected alternative" section. Thx. recasting +1 (binding) -Matthias On 5/21/18 3:04 PM, Guozhang Wang wrote: > Hello Matthias, > > I've tried it out on the

Re: [DISCUSS] KIP-298: Error Handling in Connect

2018-05-21 Thread Arjun Satish
Hey Jason, This KIP does take serialization errors to be retriable. The typical use case is that Schema Registry can have a bad/unavailable schema, which can be corrected over time. But since the converters throw DataExceptions for all failures, it is hard to determine what caused these errors.

Re: [VOTE] KIP-303: Add Dynamic Routing Support in Kafka Streams' Topology Sink

2018-05-21 Thread Guozhang Wang
Hello Matthias, I've tried it out on the PR, the implementation should be fine but one concern I had is that, as you may also realize users of DynamicStreamPartitioner needs to implement two interface functions, with and without the topic name if it is extending from StreamPartitioner; we could

[jira] [Created] (KAFKA-6928) StreamsPartitionAssignor is double retrying within InternalTopicManager

2018-05-21 Thread Guozhang Wang (JIRA)
Guozhang Wang created KAFKA-6928: Summary: StreamsPartitionAssignor is double retrying within InternalTopicManager Key: KAFKA-6928 URL: https://issues.apache.org/jira/browse/KAFKA-6928 Project: Kafka

Re: [VOTE] KIP-244: Add Record Header support to Kafka Streams

2018-05-21 Thread Jorge Esteban Quilcate Otoya
Hi, I'd like to point out that: org.apache.kafka.streams.test.ConsumerRecordFactory Has also been included as part of this KIP to support changes on test cases, just in case there is additional feedback here. Cheers, Jorge. El lun., 21 may. 2018 a las 16:46, Jorge Esteban Quilcate Otoya (<

Re: [DISCUSS] KIP-298: Error Handling in Connect

2018-05-21 Thread Jason Gustafson
Thanks Arjun. I had one additional question. High level, I'm wondering if it makes sense to treat processing errors such as serialization failures the same as IO errors. In the former case, retrying typically doesn't help because the processing is deterministic. In the latter case, the failure may

[jira] [Created] (KAFKA-6927) Broker uses significant amount of memory during down-conversion

2018-05-21 Thread Dhruvil Shah (JIRA)
Dhruvil Shah created KAFKA-6927: --- Summary: Broker uses significant amount of memory during down-conversion Key: KAFKA-6927 URL: https://issues.apache.org/jira/browse/KAFKA-6927 Project: Kafka

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

2018-05-21 Thread Apache Jenkins Server
See Changes: [github] MINOR: Reduce required occurrance from 100 to 10 (#5048) -- [...truncated 422.86 KB...] kafka.server.epoch.LeaderEpochFileCacheTest >

Re: [VOTE] KIP-290: Support for wildcard suffixed ACLs

2018-05-21 Thread Piyush Vijay
Thanks for the +1s. I'll send out a PR shortly. Thanks Piyush Vijay On Mon, May 21, 2018 at 8:59 AM, Guozhang Wang wrote: > Thanks for the KIP, +1 from me (binding). > > > Guozhang > > On Mon, May 21, 2018 at 5:22 AM, Damian Guy wrote: > > > +1

Re: [VOTE] KIP-298: Error Handling in Connect kafka

2018-05-21 Thread Arjun Satish
All, Thanks so much for your feedback on this KIP. I've made some small modifications today. I'll wait till midnight today (PDT) to close this vote. Please let me know if there are any further comments. Best, On Mon, May 21, 2018 at 11:29 AM, Ewen Cheslack-Postava wrote: >

Re: [DISCUSSION] KIP-300: Add Windowed KTable API in StreamsBuilder

2018-05-21 Thread Liquan Pei
This KIP makes sharing a WindowedKTable among Kafka Stream jobs very easy. It would be nice to get this into trunk soon. Best, Liquan On Mon, May 21, 2018 at 12:25 PM, Boyang Chen wrote: > Hey all, > > > I would like to start a discussion thread on KIP 300, which

Re: [DISCUSS] KIP-298: Error Handling in Connect

2018-05-21 Thread Arjun Satish
OK. Let's simplify tolerance to simply have NONE or ALL values. For extensions, we can open a KIP and implement in later versions. Thanks a lot! On Mon, May 21, 2018 at 1:18 PM, Ewen Cheslack-Postava wrote: > On Mon, May 21, 2018 at 12:39 PM Arjun Satish

Re: [DISCUSS] KIP-298: Error Handling in Connect

2018-05-21 Thread Ewen Cheslack-Postava
On Mon, May 21, 2018 at 12:39 PM Arjun Satish wrote: > Thanks a lot, Ewen! I'll make sure the documentation is clear on the > differences between retries an tolerance. > > Do you think percentage would have the same problem as the one you brought > up? Also, if we say 10%

Re: [DISCUSS] KIP-298: Error Handling in Connect

2018-05-21 Thread Arjun Satish
Thanks a lot, Ewen! I'll make sure the documentation is clear on the differences between retries an tolerance. Do you think percentage would have the same problem as the one you brought up? Also, if we say 10% tolerance, do we have to wait for the duration to finish before failing the task, or

[DISCUSSION] KIP-300: Add Windowed KTable API in StreamsBuilder

2018-05-21 Thread Boyang Chen
Hey all, I would like to start a discussion thread on KIP 300, which introduces a new API called windowedTable() in StreamsBuilder: https://cwiki.apache.org/confluence/display/KAFKA/KIP-300%3A+Add+Windowed+KTable+API+in+StreamsBuilder The pull request I'm working on is here:

[jira] [Resolved] (KAFKA-3665) Default ssl.endpoint.identification.algorithm should be https

2018-05-21 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3665?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma resolved KAFKA-3665. Resolution: Fixed > Default ssl.endpoint.identification.algorithm should be https >

[jira] [Reopened] (KAFKA-3665) Default ssl.endpoint.identification.algorithm should be https

2018-05-21 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3665?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma reopened KAFKA-3665: > Default ssl.endpoint.identification.algorithm should be https >

Re: [VOTE] KIP-303: Add Dynamic Routing Support in Kafka Streams' Topology Sink

2018-05-21 Thread Matthias J. Sax
I think that the risk of the change is moderate as I expect most people to use the DefaultStreamPartitioner. However, there would still be possibility to define a new interface instead of changing the old: > public interface DynamicStreamPartitioner { > Integer partition(String topic,

Re: [VOTE] KIP-303: Add Dynamic Routing Support in Kafka Streams' Topology Sink

2018-05-21 Thread Guozhang Wang
Hello everyone, While implementing the PR for this KIP I realized there is once place which we should consider modifying on public APIs as well: StreamPartitioner#partition, to add the topic name string. Note it will be a incompatible change that requires users who have customized

Re: [VOTE] KIP-298: Error Handling in Connect kafka

2018-05-21 Thread Ewen Cheslack-Postava
+1 binding. I had one last comment in the DISCUSS thread, but not really a blocker. -Ewen On Mon, May 21, 2018 at 9:48 AM Matthias J. Sax wrote: > +1 (binding) > > > > On 5/21/18 9:30 AM, Randall Hauch wrote: > > Thanks, Arjun. +1 (non-binding) > > > > Regards, > >

Re: [DISCUSS] KIP-298: Error Handling in Connect

2018-05-21 Thread Ewen Cheslack-Postava
Arjun, Understood on retries vs tolerance -- though I suspect this will end up being a bit confusing to users as well. It's two levels of error handling which is what tripped me up. One last comment on KIP (which otherwise looks good): for the tolerance setting, do we want it to be an absolute

Re: [VOTE] KIP-306: Configuration for Delaying Response to Failed Client Authentication

2018-05-21 Thread Bill Bejeck
Thanks for the KIP. +1 -Bill On Mon, May 21, 2018 at 1:41 PM, Ted Yu wrote: > +1 > > On Mon, May 21, 2018 at 9:02 AM, Guozhang Wang wrote: > > > Thanks Dhruvil for the KIP, +1 from me (binding). > > > > > > Guozhang > > > > On Mon, May 21, 2018 at

Re: [DISCUSS] KIP-298: Error Handling in Connect

2018-05-21 Thread Arjun Satish
Hey Jason, Thanks for your comments. Please find answers inline: On Mon, May 21, 2018 at 9:52 AM, Jason Gustafson wrote: > Hi Arjun, > > Thanks for the KIP. Just a few comments/questions: > > 1. The proposal allows users to configure the number of retries. I usually > find

Re: [VOTE] KIP-285: Connect Rest Extension Plugin

2018-05-21 Thread Magesh Nandakumar
Thanks a lot Guozhang. I have updated the KIP to reflect that minor change. On Mon, May 21, 2018 at 9:08 AM, Guozhang Wang wrote: > Thanks for the KIP, +1 from me (binding) > > One minor suggestion: in "class TaskState" constructor, the passed in > parameter of `worker`

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

2018-05-21 Thread Apache Jenkins Server
See Changes: [jason] KAFKA-6868; Fix buffer underflow and expose group state in the consumer -- [...truncated 423.95 KB...] kafka.zookeeper.ZooKeeperClientTest >

Re: [VOTE] KIP-306: Configuration for Delaying Response to Failed Client Authentication

2018-05-21 Thread Ted Yu
+1 On Mon, May 21, 2018 at 9:02 AM, Guozhang Wang wrote: > Thanks Dhruvil for the KIP, +1 from me (binding). > > > Guozhang > > On Mon, May 21, 2018 at 7:52 AM, Dhruvil Shah > wrote: > > > Hi, > > > > I would like to start a vote on KIP-306 which

Re: [VOTE] KIP-277 - Fine Grained ACL for CreateTopics API

2018-05-21 Thread Ismael Juma
Thanks for the KIP, +1 (binding). Can you also please describe the compatibility impact of changing the error code from CLUSTER_AUTHORIZATION_FAILED to TOPIC_AUTHORIZATION_FAILED? Ismael On Wed, Apr 25, 2018 at 2:45 AM Edoardo Comar wrote: > Hi, > > The discuss thread on

[jira] [Resolved] (KAFKA-6873) Broker is not returning data including requested offset

2018-05-21 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-6873?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma resolved KAFKA-6873. Resolution: Not A Problem > Broker is not returning data including requested offset >

Re: [DISCUSS] KIP-242: Mask password fields in Kafka Connect REST response

2018-05-21 Thread Randall Hauch
See also https://cwiki.apache.org/confluence/display/KAFKA/KIP-297%3A+Externalizing+Secrets+for+Connect+Configurations, which just passed. On Mon, Mar 19, 2018 at 11:16 PM, Ewen Cheslack-Postava wrote: > SSL authentication was added in KIP-208, which will be included in Kafka

[jira] [Resolved] (KAFKA-6839) ZK session retry with cname record

2018-05-21 Thread Tyler Monahan (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-6839?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Tyler Monahan resolved KAFKA-6839. -- Resolution: Invalid > ZK session retry with cname record > -- >

Re: [DISCUSS] KIP-298: Error Handling in Connect

2018-05-21 Thread Jason Gustafson
Hi Arjun, Thanks for the KIP. Just a few comments/questions: 1. The proposal allows users to configure the number of retries. I usually find it easier as a user to work with timeouts since it's difficult to know how long a retry might take. Have you considered adding a timeout option which would

Re: [VOTE] KIP-298: Error Handling in Connect kafka

2018-05-21 Thread Matthias J. Sax
+1 (binding) On 5/21/18 9:30 AM, Randall Hauch wrote: > Thanks, Arjun. +1 (non-binding) > > Regards, > Randall > > On Mon, May 21, 2018 at 11:14 AM, Guozhang Wang wrote: > >> Thanks for the KIP. +1 (binding) >> >> >> Guozhang >> >> On Fri, May 18, 2018 at 3:36 PM, Gwen

Re: [VOTE] KIP-277 - Fine Grained ACL for CreateTopics API

2018-05-21 Thread Edoardo Comar
Hi All I think I can close the vote as KIP-277 is approved with the following +1 votes (and no -1s) binding: Guozhang, Rajini, Damian non-binding: Ted, Stephane, Vahid, Colin, Manikumar, Matt Thanks to everyone who reviewed and voted on the KIP. On 21 May 2018 at 13:12, Damian Guy

Re: [VOTE] KIP-255: OAuth Authentication via SASL/OAUTHBEARER

2018-05-21 Thread Ron Dagostino
Thank you, Damian. This concludes the vote for KIP 255. The vote passes with three binding + 1 votes, from Rajini, Jun, and Damian, and three non-binding +1 votes, from Mickael, Manikumar, and myself. I have marked the KIP as "Accepted". The pull request is available at

Re: [VOTE] KIP-298: Error Handling in Connect kafka

2018-05-21 Thread Randall Hauch
Thanks, Arjun. +1 (non-binding) Regards, Randall On Mon, May 21, 2018 at 11:14 AM, Guozhang Wang wrote: > Thanks for the KIP. +1 (binding) > > > Guozhang > > On Fri, May 18, 2018 at 3:36 PM, Gwen Shapira wrote: > > > +1 > > > > Thank you! Error handling

Re: [VOTE] KIP-297: Externalizing Secrets for Connect Configurations

2018-05-21 Thread Gwen Shapira
+1 from me too. Can't wait to use this feature. On Mon, May 21, 2018 at 9:11 AM, Guozhang Wang wrote: > Thanks for the KIP. +1 from me (binding). > > > Guozhang > > On Fri, May 18, 2018 at 9:46 AM, Randall Hauch wrote: > > > Looks great. > > > > +1

Re: [VOTE] KIP-298: Error Handling in Connect kafka

2018-05-21 Thread Guozhang Wang
Thanks for the KIP. +1 (binding) Guozhang On Fri, May 18, 2018 at 3:36 PM, Gwen Shapira wrote: > +1 > > Thank you! Error handling in Connect will be a huge improvement. > > On Thu, May 17, 2018, 1:58 AM Arjun Satish wrote: > > > All, > > > > Many

Re: [VOTE] KIP-297: Externalizing Secrets for Connect Configurations

2018-05-21 Thread Guozhang Wang
Thanks for the KIP. +1 from me (binding). Guozhang On Fri, May 18, 2018 at 9:46 AM, Randall Hauch wrote: > Looks great. > > +1 (non-binding) > > Regards, > Randall > > On Fri, May 18, 2018 at 10:23 AM, Rajini Sivaram > wrote: > > > Thanks, Robert!

Re: [VOTE] KIP-285: Connect Rest Extension Plugin

2018-05-21 Thread Guozhang Wang
Thanks for the KIP, +1 from me (binding) One minor suggestion: in "class TaskState" constructor, the passed in parameter of `worker` could be `workerId`, to be consistent with others in ConnectorState? Guozhang On Fri, May 18, 2018 at 9:44 AM, Randall Hauch wrote: > +1

Re: [VOTE] KIP-306: Configuration for Delaying Response to Failed Client Authentication

2018-05-21 Thread Guozhang Wang
Thanks Dhruvil for the KIP, +1 from me (binding). Guozhang On Mon, May 21, 2018 at 7:52 AM, Dhruvil Shah wrote: > Hi, > > I would like to start a vote on KIP-306 which proposes to add a > configuration to delay responses to failed authentication. > > Link to the KIP: >

Re: [VOTE] KIP-290: Support for wildcard suffixed ACLs

2018-05-21 Thread Guozhang Wang
Thanks for the KIP, +1 from me (binding). Guozhang On Mon, May 21, 2018 at 5:22 AM, Damian Guy wrote: > +1 (binding) > > On Sat, 19 May 2018 at 03:51 Piyush Vijay wrote: > > > Hi everyone, > > > > I would like to start a vote for > > > >

[jira] [Created] (KAFKA-6926) Reduce NPath exceptions in Connect

2018-05-21 Thread Randall Hauch (JIRA)
Randall Hauch created KAFKA-6926: Summary: Reduce NPath exceptions in Connect Key: KAFKA-6926 URL: https://issues.apache.org/jira/browse/KAFKA-6926 Project: Kafka Issue Type: Improvement

[VOTE] KIP-306: Configuration for Delaying Response to Failed Client Authentication

2018-05-21 Thread Dhruvil Shah
Hi, I would like to start a vote on KIP-306 which proposes to add a configuration to delay responses to failed authentication. Link to the KIP: https://cwiki.apache.org/confluence/display/KAFKA/KIP-306%3A+Configuration+for+Delaying+Response+to+Failed+Client+Authentication Thanks, Dhruvil

Re: [VOTE] KIP-244: Add Record Header support to Kafka Streams

2018-05-21 Thread Jorge Esteban Quilcate Otoya
Thanks for your votes and your feedback. This KIP has been approved with the following results: Binding +1s: 3 (Matthias, Damina, Guozhang) Non-biniding +1s: (Bill, Ted) Jorge. El mar., 15 may. 2018 a las 20:01, Bill Bejeck () escribió: > Thanks for the KIP! > > +1 > >

Re: [VOTE] KIP-295: Add Streams Configuration Allowing for Optional Topology Optimization

2018-05-21 Thread Bill Bejeck
All, The 72 hours has passed so I will close this vote. KIP-295 has been accepted with 3 binding votes (Damian Guy, Matthias Sax, and Guozhang Wang) 2 non-binding (Ted Yu and myself) Thanks, Bill On Wed, May 16, 2018 at 12:20 PM, Damian Guy wrote: > +1 > On Tue, 15 May

Re: [DISCUSS] Apache Kafka 2.0.0 Release Plan

2018-05-21 Thread Jorge Esteban Quilcate Otoya
Ok, thanks! On Mon, 21 May 2018, 15:17 Rajini Sivaram, wrote: > Jorge, > > KIP-244 has binding votes from Guozhang, Matthias and Damian. So you can > close the vote and move the KIP to Accepted state. > > > On Mon, May 21, 2018 at 2:13 PM, Jorge Esteban Quilcate Otoya <

Re: [DISCUSS] Apache Kafka 2.0.0 Release Plan

2018-05-21 Thread Rajini Sivaram
Jorge, KIP-244 has binding votes from Guozhang, Matthias and Damian. So you can close the vote and move the KIP to Accepted state. On Mon, May 21, 2018 at 2:13 PM, Jorge Esteban Quilcate Otoya < quilcate.jo...@gmail.com> wrote: > I think there is one missing vote in KIP-244 too. Pull request

Re: [DISCUSS] Apache Kafka 2.0.0 Release Plan

2018-05-21 Thread Jorge Esteban Quilcate Otoya
I think there is one missing vote in KIP-244 too. Pull request is almost ready to be merged also. On Mon, 21 May 2018, 11:48 Rajini Sivaram, wrote: > Hi all, > > This is a reminder that KIP freeze for 2.0.0 release is tomorrow. KIPs that > are not yet in voting stage

Re: [VOTE] KIP-255: OAuth Authentication via SASL/OAUTHBEARER

2018-05-21 Thread Damian Guy
+1 (binding) Thanks On Mon, 21 May 2018 at 04:59 Ron Dagostino wrote: > Hi Committers. One more binding affirmative vote is required if KIP 255 > is to have a chance of being included in the 2.0.0 release. Please vote > today. > > Ron > > > On May 18, 2018, at 9:27 PM, Ron

Re: [VOTE] KIP-290: Support for wildcard suffixed ACLs

2018-05-21 Thread Damian Guy
+1 (binding) On Sat, 19 May 2018 at 03:51 Piyush Vijay wrote: > Hi everyone, > > I would like to start a vote for > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-290%3A+Support+for+wildcard+suffixed+ACLs > . > > The KIP proposes a way to support

Re: [VOTE] KIP-277 - Fine Grained ACL for CreateTopics API

2018-05-21 Thread Damian Guy
+1 (binding) Thanks On Mon, 21 May 2018 at 11:26 Edoardo Comar wrote: > Hi, KIP freeze is tomorrow. Anyone else would like to cast their vote ? > > voting so far : > binding: Guozhang, Rajini > non-binding: Ted, Stephane, Vahid, Colin, Manikumar, Matt > > On 18 May 2018 at

Re: [DISCUSS] KIP-290: Support for wildcard suffixed ACLs

2018-05-21 Thread Andy Coates
Hey Piyush, Thanks for the updated KIP! Couple of minor points from me: When storing wildcard-suffixed Acls in ZK, drop the asterisk of the end for the path, e.g. change "*/kafka-wildcard-acl/Topic/teamA*" * to "*/* *kafka-wildcard-acl**/Topic/**teamA"*. This reduces error conditions, i.e. this

Re: [DISCUSS] KIP-290: Support for wildcard suffixed ACLs

2018-05-21 Thread Ted Yu
+1 Original message From: Stephane Maarek Date: 5/21/18 3:15 AM (GMT-08:00) To: dev@kafka.apache.org Subject: Re: [DISCUSS] KIP-290: Support for wildcard suffixed ACLs +1 non binding On Mon., 21 May 2018, 2:44 pm Rajini Sivaram,

Re: [VOTE] KIP-277 - Fine Grained ACL for CreateTopics API

2018-05-21 Thread Edoardo Comar
Hi, KIP freeze is tomorrow. Anyone else would like to cast their vote ? voting so far : binding: Guozhang, Rajini non-binding: Ted, Stephane, Vahid, Colin, Manikumar, Matt On 18 May 2018 at 09:38, Edoardo Comar wrote: > bumping again as this KIP is still missing one binding

Re: [DISCUSS] KIP-290: Support for wildcard suffixed ACLs

2018-05-21 Thread Stephane Maarek
+1 non binding On Mon., 21 May 2018, 2:44 pm Rajini Sivaram, wrote: > Hi Piyush, Thanks for the KIP! > > +1 (binding) > > Regards, > > Rajini > > On Sun, May 20, 2018 at 2:53 PM, Andy Coates wrote: > > > Awesome last minute effort Piyush. > > > >

RE: [VOTE] KIP-235 Add DNS alias support for secured connection

2018-05-21 Thread Skrzypek, Jonathan
Hi, What would be the next step here ? I know there's a discussion going on around KIP-302, but I'm also conscious that the 2.0.0 deadline for KIPs is tomorrow. I've opened this KIP in January and discussions have been productive with an end solution I had the impression was reasonable, so I am

Re: [DISCUSS] Apache Kafka 2.0.0 Release Plan

2018-05-21 Thread Rajini Sivaram
Hi all, This is a reminder that KIP freeze for 2.0.0 release is tomorrow. KIPs that are not yet in voting stage will be postponed to the next release. We have several KIPs that need more votes to be accepted. Please participate in reviews and votes to enable these to be added to the release (or

[jira] [Created] (KAFKA-6925) Memory leak in org.apache.kafka.streams.processor.internals.StreamThread$StreamsMetricsThreadImpl

2018-05-21 Thread Marcin Kuthan (JIRA)
Marcin Kuthan created KAFKA-6925: Summary: Memory leak in org.apache.kafka.streams.processor.internals.StreamThread$StreamsMetricsThreadImpl Key: KAFKA-6925 URL: https://issues.apache.org/jira/browse/KAFKA-6925

Re: [VOTE] KIP-278: Add version option to Kafka's commands

2018-05-21 Thread Rajini Sivaram
Hi Sasaki, Thanks for the KIP! +1 (binding) On Mon, May 14, 2018 at 4:30 PM, Sasaki Toru wrote: > Rajini, > > Thank you for your appropriate advice, I misunderstood some non-binding > votes. > I get this KIP back to the state waiting to vote. > > > Thanks, > Sasaki >

Re: [DISCUSS] KIP-290: Support for wildcard suffixed ACLs

2018-05-21 Thread Rajini Sivaram
Hi Piyush, Thanks for the KIP! +1 (binding) Regards, Rajini On Sun, May 20, 2018 at 2:53 PM, Andy Coates wrote: > Awesome last minute effort Piyush. > > Really appreciate your time and input, > > Andy > > Sent from my iPhone > > > On 19 May 2018, at 03:43, Piyush Vijay

Kafka consumer to unzip stream of .gz files and read

2018-05-21 Thread mayur shah
HI Team, Greeting! I am facing one issue on kafka consumer using python hope you guys help us to resolve this issue Kafka consumer to unzip stream of .gz files and read Kafka producer is sending

Re: [VOTE] #2 KIP-248: Create New ConfigCommand That Uses The New AdminClient

2018-05-21 Thread Viktor Somogyi
Hi All, I'd like to ask the community to please vote for this as the KIP freeze is tomorrow. Thank you very much, Viktor On Mon, May 21, 2018 at 9:39 AM, Viktor Somogyi wrote: > Hi Colin, > > Sure, I'll add a note. > Thanks for your vote. > > Viktor > > On Sat, May 19,

Re: [VOTE] #2 KIP-248: Create New ConfigCommand That Uses The New AdminClient

2018-05-21 Thread Viktor Somogyi
Hi Colin, Sure, I'll add a note. Thanks for your vote. Viktor On Sat, May 19, 2018 at 1:01 AM, Colin McCabe wrote: > Hi Viktor, > > Thanks, this looks good. > > The boolean should default to false if not set, to ensure that existing > clients continue to work as-is, right?