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

2018-05-16 Thread Rajini Sivaram
Colin/Piyush, I was under the impression that we are keeping both Resource and ResourceFilter since they are used in different contexts. And hence we would retain both AclBinding and AclBindingFilter. Is that not the case? Anyway, an updated KIP would be useful since it has got to a stage where

Re: [DISCUSS] KIP-303: Add Dynamic Routing in Streams Sink

2018-05-16 Thread Guozhang Wang
I have thought about exposing record context as well, and in the end I decided to piggy-back it with KIP-159. And if we want to indeed reuse the class it will be: ``` public interface RichKeyValueMapper { VR apply(final K key, final V value, final RecordContext recordContext); } ```

Re: [DISCUSS] KIP-302 - Enable Kafka clients to use all DNS resolved IP addresses

2018-05-16 Thread Edoardo Comar
Hi Rajini, In your example KIP-302 would attempt to connect to the first address returned, let's say www.apache.org/195.154.151.36 then, only if that fails, will in turn try the remaining: www.apache.org/40.79.78.1 www.apache.org/140.211.11.105 www.apache.org/2001:bc8:2142:300:0:0:0:0 You're

[jira] [Created] (KAFKA-6909) Message Accumulation in case of no connection to Kafka

2018-05-16 Thread Alexander Stepanov (JIRA)
Alexander Stepanov created KAFKA-6909: - Summary: Message Accumulation in case of no connection to Kafka Key: KAFKA-6909 URL: https://issues.apache.org/jira/browse/KAFKA-6909 Project: Kafka

Re: Permission request to open a KIP

2018-05-16 Thread Matthias J. Sax
Done. On 5/16/18 7:55 AM, Saulius Valatka wrote: > Hi, > > could someone please grant me permission to open a KIP? My wiki id is > saulius.vl > > Thanks > signature.asc Description: OpenPGP digital signature

[jira] [Created] (KAFKA-6910) Ability to specify a default state store type or factory

2018-05-16 Thread Antony Stubbs (JIRA)
Antony Stubbs created KAFKA-6910: Summary: Ability to specify a default state store type or factory Key: KAFKA-6910 URL: https://issues.apache.org/jira/browse/KAFKA-6910 Project: Kafka Issue

Permission request to open a KIP

2018-05-16 Thread Saulius Valatka
Hi, could someone please grant me permission to open a KIP? My wiki id is saulius.vl Thanks

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

2018-05-16 Thread Robert Yokota
Hi Rajini, Thanks for the excellent feedback! I've made the API changes that you've requested in the KIP. > 1. Are we expecting one provider instance with different contexts > provided to `ConfigProvider.get()`? If we created a different provider > instance for each context, we could deal with

Re: Kafka system tests contribution

2018-05-16 Thread Andriy Sorokhtey
Hi, Did anyone had a chance to take a look at this issue? 2018-05-08 15:01 GMT+03:00 Andriy Sorokhtey : > Hello Kafka team > > I’d like to contribute to the Kafka system tests. > > I’ve tried to execute system tests locally and I have some issues. Can > anyone give me a

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

2018-05-16 Thread Viktor Somogyi
Hi Colin, > Doing get-merge-set is buggy, though. If someone else does get-merge-set at > the same time as you, you might overwrite that person's changes, or vice > versa. So I really don't think we should try to do this. Also, having both > an incremental and a full API is useful, and it's

[jira] [Resolved] (KAFKA-6908) Update LogDirsCommand's prompt information

2018-05-16 Thread darion yaphet (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-6908?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] darion yaphet resolved KAFKA-6908. -- Resolution: Won't Fix > Update LogDirsCommand's prompt information >

[jira] [Created] (KAFKA-6908) Update LogDirsCommand's prompt information

2018-05-16 Thread darion yaphet (JIRA)
darion yaphet created KAFKA-6908: Summary: Update LogDirsCommand's prompt information Key: KAFKA-6908 URL: https://issues.apache.org/jira/browse/KAFKA-6908 Project: Kafka Issue Type:

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

2018-05-16 Thread Rajini Sivaram
Hi Robert, Thanks for the KIP updates. The interfaces look suitable for brokers, with some small changes. If we can adapt the interface to implement the existing DynamicBrokerConfig, then we are good. With broker configs: 1. We don't know what configs are in ZK since we allow custom

[jira] [Created] (KAFKA-6907) Not able to delete topic

2018-05-16 Thread praveen (JIRA)
praveen created KAFKA-6907: -- Summary: Not able to delete topic Key: KAFKA-6907 URL: https://issues.apache.org/jira/browse/KAFKA-6907 Project: Kafka Issue Type: Bug Components: config

Re: [VOTE] KIP-282: Add the listener name to the authentication context

2018-05-16 Thread Mickael Maison
Thanks to everyone who voted and reviewed the KIP. The vote has passed with 3 binding votes (Rajini, Jun and Ismael) and 2 non-binding votes (Ted and Manikumar). Link to the PR: https://github.com/apache/kafka/pull/4829 On Tue, May 15, 2018 at 7:05 PM, Ismael Juma wrote: >

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

2018-05-16 Thread Mickael Maison
Thanks for the KIP, +1 (non binding) On Wed, May 16, 2018 at 2:51 AM, Ron Dagostino wrote: > Hi Jun. I think you are getting at the fact that OAuth 2 is a flexible > framework that allows different installations to do things differently. It > is true that the principal name

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

2018-05-16 Thread Damian Guy
+1 On Tue, 15 May 2018 at 15:04, Ted Yu wrote: > +1 > Original message From: Guozhang Wang > Date: 5/15/18 2:34 PM (GMT-08:00) To: dev@kafka.apache.org Subject: Re: > [VOTE] KIP-295: Add Streams Configuration Allowing for Optional

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

2018-05-16 Thread Rajini Sivaram
Thanks for the update, Robert. Looks good to me. Regards, Rajini On Wed, May 16, 2018 at 4:43 PM, Robert Yokota wrote: > Hi Rajini, > > Thanks for the excellent feedback! > > I've made the API changes that you've requested in the KIP. > > > > 1. Are we expecting one

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

2018-05-16 Thread Manikumar
+1 (non-binding) Thanks for the detailed KIP. On Wed, May 16, 2018 at 5:59 PM, Mickael Maison wrote: > Thanks for the KIP, > +1 (non binding) > > On Wed, May 16, 2018 at 2:51 AM, Ron Dagostino wrote: > > Hi Jun. I think you are getting at the fact

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

2018-05-16 Thread Konstantine Karantasis
Arjun, it's exciting to see a KIP around better handling of bad-data and errors in Kafka Connect. I have only a few comments below, which I hope you'll find helpful. 1. I think it'd be useful to describe a bit more in detail how someone can extract the raw data of a Kafka record that failed to

RE: [DISCUSS] KIP-302 - Enable Kafka clients to use all DNS resolved IP addresses

2018-05-16 Thread Skrzypek, Jonathan
I think there are combinations that will break SASL and SSL auth. Could the trick be to have a single parameter that triggers dns resolve both for bootstrap and advertised listeners, both using getCanonicalHostName() ? Jonathan Skrzypek -Original Message- From: Edoardo Comar

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

2018-05-16 Thread Magesh Nandakumar
Ewen, Thanks for your comments. I have made the changes to the package names and also moved the nested class up in the package. Public API would include *org.apache.kafka.connect.rest* -ConnectClusterState -ConnectRestExtension -ConnectRestExtensionContext *org.apache.kafka.connect.health*

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

2018-05-16 Thread Ewen Cheslack-Postava
Hey, Sorry for the late follow up. I just had a couple of minor questions about details: * Some of the public API being added is under a runtime package. But that would be new for public API -- currently only things under the runtime package use that package name. I think changing the package

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

2018-05-16 Thread Apache Jenkins Server
See Changes: [github] MINOR: add missing parameter `processing.guaratees` to Streams docs -- [...truncated 420.14 KB...] kafka.admin.ResetConsumerGroupOffsetTest >

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

2018-05-16 Thread Apache Jenkins Server
See Changes: [wangguoz] KAFKA-6905: Document that Processors may be re-used by Streams (#5022) [github] MINOR: fix broken links in streams doc (#5025) [github] MINOR: add missing parameter

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

2018-05-16 Thread Randall Hauch
A few very minor suggestions: 1. There are a few formatting issues with paragraphs that use a monospace font. Minor, but it would be nice to fix. 2. Would be nice to link to the PR 3. Do we need the org.apache.kafka.connect.rest.extension.entities package? Could we just move the

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

2018-05-16 Thread Robert Yokota
Hello everyone, After a good round of discussions with excellent feedback and no major objections, I would like to start a vote on KIP-297 to externalize secrets from Kafka Connect configurations. My thanks in advance! KIP: <

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

2018-05-16 Thread Jun Rao
Hi, Ron, Thanks. I understand now. It may be useful to add a reference to JWT in the KIP. Jun On Tue, May 15, 2018 at 6:51 PM, Ron Dagostino wrote: > Hi Jun. I think you are getting at the fact that OAuth 2 is a flexible > framework that allows different installations to

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

2018-05-16 Thread Magesh Nandakumar
Randall, I have adjusted the package names per Ewen's suggestions and also made some minor edits per your suggestions. Since there are no major outstanding issues, i'm moving this to vote. Thanks Magesh On Wed, May 16, 2018 at 4:38 PM, Randall Hauch wrote: > A few very minor

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

2018-05-16 Thread Colin McCabe
Thanks, Robert. Looks good overall. As a clarification about the indirections, what if I have the connect configuration key foo set up as ${vault:bar}, and in Vault, have the bar key set to ${file:baz}? Does connect get foo as the contents of the baz file? I would argue that it should not

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

2018-05-16 Thread Arjun Satish
Hi Konstantine, Thanks a lot for your feedback. I have made the necessary changes to the KIP. Best, On Wed, May 16, 2018 at 11:38 AM, Konstantine Karantasis < konstant...@confluent.io> wrote: > Arjun, it's exciting to see a KIP around better handling of bad-data and > errors in Kafka Connect.

[VOTE] KIP-285: Connect Rest Extension Plugin

2018-05-16 Thread Magesh Nandakumar
Hello everyone, After a good round of discussions with excellent feedback and no major objections, I would like to start a vote on KIP-285: Connect Rest Extension Plugin. KIP: < https://cwiki.apache.org/confluence/display/KAFKA/KIP-285%3A+Connect+Rest+Extension+Plugin > JIRA:

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

2018-05-16 Thread Apache Jenkins Server
See Changes: [github] MINOR: fix broken links in streams doc (#5025) [github] MINOR: add missing parameter `processing.guaratees` to Streams docs -- [...truncated 3.57

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

2018-05-16 Thread Randall Hauch
Looks good to me. Thanks for quickly making the changes! Great work! Best regards, Randall > On May 16, 2018, at 8:07 PM, Magesh Nandakumar wrote: > > Randall, > > I have adjusted the package names per Ewen's suggestions and also made some > minor edits per your

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

2018-05-16 Thread Matt Farmer
Hey Arjun, I like deadletterqueue all lower case, so I'm +1 on that. Yes, in the case we were seeing there were external system failures. We had issues connecting to S3. While the connector does include some retry functionality, however setting these values sufficiently high seemed to cause us

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

2018-05-16 Thread Robert Yokota
Hi Colin, Good questions. > As a clarification about the indirections, what if I have the connect configuration key foo set up as ${vault:bar}, and in Vault, have the bar key set to ${file:baz}? > Does connect get foo as the contents of the baz file? I would argue that it should not (and in

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

2018-05-16 Thread Magesh Nandakumar
Arjun, Thanks for all the changes. Technically, the message format used for the DLQ should be part of the public interface since users could consume it and take actions. Thanks, Magesh On Wed, May 16, 2018 at 6:56 PM, Arjun Satish wrote: > Hi Konstantine, > > Thanks a

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

2018-05-16 Thread Matt Farmer
+1 (non-binding) On Tue, May 15, 2018 at 4:26 AM, Edoardo Comar wrote: > Hi, > bumping the thread as the current vote count for this KIP is > 2 binding +1 > 5 non-binding +1 > > thanks, Edo > > On 8 May 2018 at 16:14, Edoardo Comar wrote: > > Hi, > >

Re: [DISCUSS] KIP-303: Add Dynamic Routing in Streams Sink

2018-05-16 Thread Damian Guy
Overall i'm a +1 on this, but i'm not a big fan of using the KeyValueMapper to choose the topic. It is a bit counter-intuitve to me. I'd prefer to add a class specifically for it and possibly pass in the RecordContext On Wed, 16 May 2018 at 13:22 Guozhang Wang wrote: > Hello

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

2018-05-16 Thread Jakub Scholz
Hi, What do you plan to use the read-only REST interface for? Is there something what you cannot get through metrics interface? Otherwise it might be easier to just disable the REST interface (either in the code, or just on the platform level - e.g. in Kubernetes). Also, I do not know what is

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

2018-05-16 Thread Apache Jenkins Server
See Changes: [wangguoz] KAFKA-6905: Document that Processors may be re-used by Streams (#5022) -- [...truncated 424.29 KB...] kafka.admin.ResetConsumerGroupOffsetTest >

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

2018-05-16 Thread Colin McCabe
> On Tue, May 15, 2018 at 7:18 PM, Rajini Sivaram > wrote: > > > Hi Piyush, > > > > It is possible to configure PrincipalBuilder for SASL ( > > https://cwiki.apache.org/confluence/display/KAFKA/KIP- > >

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

2018-05-16 Thread Apache Jenkins Server
See Changes: [wangguoz] KAFKA-6905: Document that Processors may be re-used by Streams (#5022) [github] MINOR: fix broken links in streams doc (#5025) -- [...truncated

Re: Kafka Elasticsearch Connector

2018-05-16 Thread Rahul Singh
Which method did you use the most. What exactly are you having a problem with. -- Rahul Singh rahul.si...@anant.us Anant Corporation On May 15, 2018, 7:44 AM -0500, Raj, Gokul (External) , wrote: > Hi Team, > Am new to this tech. I need to connect Kafka to Elastic

Re: [VOTE] KIP-283: Efficient Memory Usage for Down-Conversion

2018-05-16 Thread Dhruvil Shah
Thanks, Ismael. I added the "Testing Strategy" section to the KIP outlining the findings. I am also closing this vote with 3 binding and 1 non-binding +1s and no objections. Thanks everyone for your review and feedback. - Dhruvil On Tue, May 15, 2018 at 11:04 AM, Ismael Juma

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

2018-05-16 Thread Saulius Valatka
Hi, I'd like to start a discussion on the following KIP: https://cwiki.apache.org/confluence/display/KAFKA/KIP-304%3A+Connect+runtime+mode+improvements+for+container+platforms Basically the idea is to make it easier to run separate instances of Kafka Connect hosting isolated connectors on

[jira] [Resolved] (KAFKA-6905) Document that Processor objects can be reused

2018-05-16 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-6905?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang resolved KAFKA-6905. -- Resolution: Fixed Fix Version/s: 2.0.0 > Document that Processor objects can be reused >

Re: [DISCUSS] KIP-303: Add Dynamic Routing in Streams Sink

2018-05-16 Thread Guozhang Wang
Hello folks, Please let me know if you have further feedbacks; if there is no more feedbacks I'm going to start the voting thread soon. Guozhang On Wed, May 16, 2018 at 8:31 AM, Guozhang Wang wrote: > I have thought about exposing record context as well, and in the end I

Re: [DISCUSS] KIP-302 - Enable Kafka clients to use all DNS resolved IP addresses

2018-05-16 Thread Edoardo Comar
Hi Jonathan, I am afraid that may not work for everybody. It would not work for us. With our current DNS, my Kafka clients are perfectly happy to use any IPs - DNS has multiple A records for the 'myhostname.mydomain' used for bootstrap and advertised listeners. The hosts all serve TLS

[jira] [Resolved] (KAFKA-6838) Transaction timeout after punctuation results in ProducerFencedException

2018-05-16 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-6838?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax resolved KAFKA-6838. Resolution: Duplicate Thanks for confirmation [~feli6]! Closing as duplicate. >

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

2018-05-16 Thread Colin McCabe
Thanks, Robert. With regard to delayMs, can’t we just restart the Connector when the keys are actually changed? Or is the concern that this would lengthen the effective key rotation time? Can’t the user just configure a slightly shorter key rotation time to counteract this concern? Regards,

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

2018-05-16 Thread Colin McCabe
Hi Viktor, The shell command isn’t that easy to integrate into applications. AdminClient will get integrated into a lot more stuff, which increases the potential for conflicts. I would argue that we should fix this soon. If we do want to reduce the scope in this KIP, we could do the merge in

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

2018-05-16 Thread Robert Yokota
Hi Colin, > With regard to delayMs, can’t we just restart the > Connector when the keys are actually changed? Currently the VaultConfigProvider does not find out when values for keys have changed. You could do this with a poll model (with a background thread in the ConfigProvider), but since

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

2018-05-16 Thread Apache Jenkins Server
See