Re: [DISCUSS] KIP-273 Kafka to support using ETCD beside Zookeeper

2018-05-14 Thread Molnár Bálint
Hi Colin, Do you have a rough estimate on that? Thanks, Balint Colin McCabe ezt írta (időpont: 2018. máj. 9., Sze, 19:53): > Hi Molnar, > > The points Ismael brought up earlier (and that were brought up on KIP-30) > are still relevant here. As Ismael said, the goal is to get rid of > external

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

2018-05-14 Thread Sasaki Toru
Hi all, This KIP would be accepted with 4 binding and 4 non-binding votes. Thank you for participating. Thanks, Sasaki. 2018-05-12 9:31 GMT+09:00 Colin McCabe : +1 (non-binding) Colin On Fri, May 11, 2018, at 12:35, Attila Sasvári wrote: +1 (non-binding) Thomas Crayford ezt írta (időpo

Re: [VOTE] KIP-294 - Enable TLS hostname verification by default

2018-05-14 Thread Rajini Sivaram
The vote has passed with three binding (Jun, Ismael, me) and five non-binding (Ted, Michael, Manikumar, Edoardo, Jakub) votes. I will update the KIP page. Thanks everyone! Regards, Rajini On Fri, May 11, 2018 at 2:28 PM, Rajini Sivaram wrote: > Hi Jun, > > I have updated the KIP with examples

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

2018-05-14 Thread Skrzypek, Jonathan
Up :) Anyone for a binding vote here ? Jonathan Skrzypek -Original Message- From: Rajini Sivaram [mailto:rajinisiva...@gmail.com] Sent: 10 May 2018 13:17 To: dev Subject: Re: [VOTE] KIP-235 Add DNS alias support for secured connection Thanks Jonathan. You have binding votes from me and

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

2018-05-14 Thread Rajini Sivaram
Sasaki, I believe this has 2 binding votes and six non-binding votes, so needs another binding vote to pass. On Mon, May 14, 2018 at 9:00 AM, Sasaki Toru wrote: > Hi all, > > This KIP would be accepted with 4 binding and 4 non-binding votes. > > Thank you for participating. > > > Thanks, > Sasa

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

2018-05-14 Thread Ismael Juma
Thanks for the KIP, Jonathan. It would be helpful to have more detail on how SSL authentication could be broken if the new behaviour is the default. I know this was discussed in the mailing list thread, but it's important to include it in the KIP since it's the main reason why a new config is neede

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

2018-05-14 Thread Mickael Maison
+1 (non binding) Thanks for the KIP On Mon, May 14, 2018 at 10:36 AM, Rajini Sivaram wrote: > Sasaki, > > I believe this has 2 binding votes and six non-binding votes, so needs > another binding vote to pass. > > On Mon, May 14, 2018 at 9:00 AM, Sasaki Toru > wrote: > >> Hi all, >> >> This KIP w

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

2018-05-14 Thread Skrzypek, Jonathan
Sure, I modified the KIP to add more details https://cwiki.apache.org/confluence/display/KAFKA/KIP-235%3A+Add+DNS+alias+support+for+secured+connection Jonathan Skrzypek -Original Message- From: Ismael Juma [mailto:ism...@juma.me.uk] Sent: 14 May 2018 11:53 To: dev Subject: Re: [VOTE

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

2018-05-14 Thread Edoardo Comar
Hi all, We just opened a KIP to add support for the client to use all IPs returned by DNS for the brokers The details are here - https://cwiki.apache.org/confluence/display/KAFKA/KIP-302+-+Enable+Kafka+clients+to+use+all+DNS+resolved+IP+addresses The JIRA and provisional PR (where the discussi

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

2018-05-14 Thread Skrzypek, Jonathan
Hi, I see you noted the similarities with KIP-235. But KIP-235 might also solve what this KIP is trying to achieve. When parsing bootstrap.servers, KIP-235 has the client add all underlying hostnames and IPs. And this happens before hitting the NetworkClient. So to me the client will try every

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

2018-05-14 Thread Skrzypek, Jonathan
Ah, apologies didn't see there was already a decent amount of discussion on this in the PR. This kind of sounds related to the environment you're running to me. What is the rationale behind using the advertised listeners to do your load balancing advertisement rather than a top level alias that

Exactly Once, Producer configuration

2018-05-14 Thread Thomas.Kass
Hi, the following article describes how to configure Apache Kafka system to enforce exactly once approach: https://www.confluent.io/blog/exactly-once-semantics-are-possible-heres-how-apache-kafka-does-it/ One recommendation states that parameter transactional.id should be set in producer.prope

[jira] [Created] (KAFKA-6900) Add thenCompose to KafkaFuture

2018-05-14 Thread Richard Tjerngren (JIRA)
Richard Tjerngren created KAFKA-6900: Summary: Add thenCompose to KafkaFuture Key: KAFKA-6900 URL: https://issues.apache.org/jira/browse/KAFKA-6900 Project: Kafka Issue Type: Improvement

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

2018-05-14 Thread Rajini Sivaram
Hi Edo, Thanks for the KIP. I think it will be good to include a diagram to make it easier to distinguish this scenario from that of KIP-235 without reading the PR. It may be worth considering if KIP-235 and this KIP could use a single config name with different values instead of two boolean conf

[jira] [Created] (KAFKA-6901) Kafka crashes when trying to delete segment when retetention time is reached

2018-05-14 Thread JIRA
Grégory R. created KAFKA-6901: - Summary: Kafka crashes when trying to delete segment when retetention time is reached Key: KAFKA-6901 URL: https://issues.apache.org/jira/browse/KAFKA-6901 Project: Kafka

KAFKA-6864

2018-05-14 Thread git
Hi there, could anyone please have a look at * https://issues.apache.org/jira/browse/KAFKA-6864 and the pull request as well as * https://github.com/apache/kafka/pull/3873 ? Thanks, Mathias

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

2018-05-14 Thread Sasaki Toru
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 2018-05-14 20:03 GMT+09:00 Mickael Maison : +1 (non binding) Thanks for the KIP On Mon, May 14, 2018 at 10:36 AM, Rajini Sivaram wrote:

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

2018-05-14 Thread Jorge Esteban Quilcate Otoya
Thanks for your feedback everyone! If there is no more comments on this KIP, I think we can open the VOTE thread. Cheers, Jorge. El sáb., 12 may. 2018 a las 2:02, Guozhang Wang () escribió: > Yeah I'm only talking about the DSL part (i.e. how stateful / stateless > operators default inheritance

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

2018-05-14 Thread Jorge Esteban Quilcate Otoya
Hi everyone, I would like to start a vote on KIP-244: Add Record Header support to Kafka Streams KIP wiki page: https://cwiki.apache.org/confluence/display/KAFKA/KIP-244%3A+Add+Record+Header+support+to+Kafka+Streams+Processor+API The discussion thread is here: http://mail-archives.apache.org/mod

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

2018-05-14 Thread Ted Yu
+1 On Mon, May 14, 2018 at 8:31 AM, Jorge Esteban Quilcate Otoya < quilcate.jo...@gmail.com> wrote: > Hi everyone, > > I would like to start a vote on KIP-244: Add Record Header support to Kafka > Streams > > KIP wiki page: > https://cwiki.apache.org/confluence/display/KAFKA/KIP- > 244%3A+Add+Rec

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

2018-05-14 Thread Guozhang Wang
Thanks Jorge, that sounds good to me. Also please feel free to send out the PR for reviews while the KIP is being voted on. Guozhang On Mon, May 14, 2018 at 8:31 AM, Jorge Esteban Quilcate Otoya < quilcate.jo...@gmail.com> wrote: > Thanks for your feedback everyone! > > If there is no more co

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

2018-05-14 Thread Edoardo Comar
Hi Jonathan thanks for your comments > This kind of sounds related to the environment you're running to me. > What is the rationale behind using the advertised listeners to do your load > balancing advertisement rather than a top level alias that has everything ? > > It sounds like in your case t

59 Kafka stream tests fails with one error...

2018-05-14 Thread Wabi SA
Want to fix this error but need community help because I am not java man. Thanks in advance, see logs please. Tests fails with next error: [INFO:2018-05-14 18:48:51,676]: starting test run with session id 2018-05-14--008... [INFO:2018-05-14 18:48:51,676]: running 1 tests... [INFO:2018-05-14 18:48:

Re: [VOTE] KIP-292: Add KTable#transformValues() method in Kafka Streams DSL

2018-05-14 Thread Damian Guy
+1 On Fri, 11 May 2018 at 14:30 Matthias J. Sax wrote: > +1 (binding) > > > On 5/11/18 2:17 PM, Ted Yu wrote: > > +1 > > > > On Fri, May 11, 2018 at 2:16 PM, Guozhang Wang > wrote: > > > >> +1 (binding) > >> > >> Also slightly changed the title beyond changing [DISCUSS] to [VOTE] > since > >> G

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

2018-05-14 Thread Jorge Esteban Quilcate Otoya
Yes, I've one already created: https://github.com/apache/kafka/pull/4955 On Mon, 14 May 2018, 17:55 Guozhang Wang, wrote: > Thanks Jorge, that sounds good to me. > > Also please feel free to send out the PR for reviews while the KIP is being > voted on. > > > Guozhang > > > On Mon, May 14, 2018

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

2018-05-14 Thread Edoardo Comar
Thanks Rajini I still don't see the overlap between the two KIPS KIP-235 allows an expansion of hostnames on the bootstrap list. KIP-302 allows alternative IPs to be used to attempt a connection (either at bootstrap and when processing the MetadataResponse) to a given hostname. A use case would

Re: [VOTE] KIP-292: Add KTable#transformValues() method in Kafka Streams DSL

2018-05-14 Thread Andy Coates
Hey All, I've updated the KIP to also cover the Scala API for completeness. (The PR already covered this). Does this require another vote? On 11 May 2018 at 14:29, Matthias J. Sax wrote: > +1 (binding) > > > On 5/11/18 2:17 PM, Ted Yu wrote: > > +1 > > > > On Fri, May 11, 2018 at 2:16 PM, Guoz

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

2018-05-14 Thread Andy Coates
+1 On 11 May 2018 at 17:14, Colin McCabe wrote: > Hi Andy, > > I see what you mean. I guess my thought here is that if the fields are > private, we can change it later if we need to. > > I definitely agree that we should use the scheme you describe for sending > ACLs over the wire (just the str

Re: [VOTE] KIP-292: Add KTable#transformValues() method in Kafka Streams DSL

2018-05-14 Thread Guozhang Wang
Thanks Andy. That's fine, we can continue the vote thread. On Mon, May 14, 2018 at 9:25 AM, Andy Coates wrote: > Hey All, > > I've updated the KIP to also cover the Scala API for completeness. (The PR > already covered this). > > Does this require another vote? > > On 11 May 2018 at 14:29, Matth

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

2018-05-14 Thread Guozhang Wang
+1 from me One more comment on the wiki: while reviewing the PR I realized that in ` MockProcessorContext.java ` we are also adding one additional API plus modifying the existing `setRecordMetadata` API. Since t

[jira] [Created] (KAFKA-6902) During replica reassignment if source broker is down the partition directory may not be removed from broker log.dirs

2018-05-14 Thread Koelli Mungee (JIRA)
Koelli Mungee created KAFKA-6902: Summary: During replica reassignment if source broker is down the partition directory may not be removed from broker log.dirs Key: KAFKA-6902 URL: https://issues.apache.org/jira/b

[DISCUSS] KIP-301 Schema Inferencing in JsonConverters

2018-05-14 Thread Allen Tang
Hi, I just opened a KIP to add Schema Inferencing in JsonConverters for Kafka Connect. The details of the proposal can be found here: https://cwiki.apache.org/confluence/display/KAFKA/KIP-301%3A+Schema+Inferencing+for+JsonConverter Also, I have created a - 1.) JIRA ticket: https://issues.apache

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

2018-05-14 Thread Matt Farmer
Hi Arjun, I'm following this very closely as better error handling in Connect is a high priority for MailChimp's Data Systems team. A few thoughts (in no particular order): For the dead letter queue configuration, could we use deadLetterQueue instead of dlq? Acronyms are notoriously hard to keep

Re: [VOTE] KIP-275 - Indicate "isClosing" in the SinkTaskContext

2018-05-14 Thread Matt Farmer
Bumping this thread. For anyone who needs a refresher the discussion thread is here: http://mail-archives.apache.org/mod_mbox/kafka-dev/201803.mbox/%3CCAM5dya9x---9M3uEf_wrJL5dw%2B6HLV4%3D5PfKKSTPE1vOHEWC_g%40mail.gmail.com%3E And there's a work in progress PR open here: https://github.com/apache

[jira] [Created] (KAFKA-6903) Improve KTable's sending old value behavior

2018-05-14 Thread Guozhang Wang (JIRA)
Guozhang Wang created KAFKA-6903: Summary: Improve KTable's sending old value behavior Key: KAFKA-6903 URL: https://issues.apache.org/jira/browse/KAFKA-6903 Project: Kafka Issue Type: Improve

Re: [VOTE] KIP-292: Add KTable#transformValues() method in Kafka Streams DSL

2018-05-14 Thread Andy Coates
Hey all, With 3 binding votes and 1 non-binding, this KIP has been accepted. Thanks for everyones input and time. Andy On 14 May 2018 at 09:42, Guozhang Wang wrote: > Thanks Andy. That's fine, we can continue the vote thread. > > On Mon, May 14, 2018 at 9:25 AM, Andy Coates wrote: > > > He

[jira] [Resolved] (KAFKA-5907) Support aggregatedJavadoc in Java 9

2018-05-14 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5907?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma resolved KAFKA-5907. Resolution: Fixed [~omkreddy] It does seem to work now, so will mark it as resolved. > Support aggr

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

2018-05-14 Thread Jun Rao
Hi, Ron, Thanks for the KIP. +1 from me. Just a few minor comments below. 1. It seems that we can translate an OAuth token to a principle name through the claim name configured in JASS. However, it's not clear to me how an OAuth token is mapped to a claim. Could you clarify that? 2. The wiki has

[jira] [Created] (KAFKA-6904) DynamicBrokerReconfigurationTest#testAdvertisedListenerUpdate is flaky

2018-05-14 Thread Ted Yu (JIRA)
Ted Yu created KAFKA-6904: - Summary: DynamicBrokerReconfigurationTest#testAdvertisedListenerUpdate is flaky Key: KAFKA-6904 URL: https://issues.apache.org/jira/browse/KAFKA-6904 Project: Kafka Issue

Jenkins build is back to normal : kafka-0.11.0-jdk7 #368

2018-05-14 Thread Apache Jenkins Server
See

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

2018-05-14 Thread Jun Rao
Hi, Mickael, Thanks for the KIP. +1 Jun On Wed, Apr 25, 2018 at 1:51 AM, Mickael Maison wrote: > Hi, > > There has been no objections in the DISCUSS thread so I'd like to > start a vote on KIP-282: > https://cwiki.apache.org/confluence/display/KAFKA/KIP- > 282%3A+Add+the+listener+name+to+the+a

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

2018-05-14 Thread Piyush Vijay
Colin, createAcls take a AclBinding, however, instead of AclBindingFilter. What are your thoughts here? public abstract DescribeAclsResult describeAcls(AclBindingFilter filter, DescribeAclsOptions options); public abstract CreateAclsResult createAcls(Collection acls, CreateAclsOptions options);

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

2018-05-14 Thread Ron Dagostino
Hi Jun. Thanks for the +1 vote. Regarding the first question about token claims, yes, you have it correct about translating the OAuth token to a principle name via a JAAS module option in the default unsecured case. Specifically, the OAuth SASL Server implementation is responsible for setting th

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

2018-05-14 Thread Matthias J. Sax
+1 (binding) Thanks a lot for the KIP! -Matthias On 5/14/18 10:17 AM, Guozhang Wang wrote: > +1 from me > > One more comment on the wiki: while reviewing the PR I realized that in ` > MockProcessorContext.java >

Re: Can anyone take a look at this KIP and Jira?

2018-05-14 Thread Matthias J. Sax
I agree with Ted. This change does not require a KIP. If ok with you, I can delete the KIP page in the Wiki for you. -Matthias On 5/10/18 11:59 AM, Ted Yu wrote: > Since the change is internal to *SensorAccess class, looks like KIP is not > required.* > > On Thu, May 10, 2018 at 11:54 AM, qingj

Re: Kafka system tests contribution

2018-05-14 Thread Matthias J. Sax
Hard to say why it fails. Note, that some system tests a flaky (we are working constantly to stabilize them) What helps debugging is to look into the ducktape log itself. You can find it in the file `test_log.debug` that is written into `resutls/` Also inspect the actual test log files, as we

Re: Exactly Once, Producer configuration

2018-05-14 Thread Matthias J. Sax
Hi, I don't think that the `producer.properties` file exist once per broker, because this file should not be at the broker machine at all. It should be on the application side (ie, your "modules"). For transactions to work correctly, each `Producer` instance needs a cluster wide unique transactio

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

2018-05-14 Thread Guozhang Wang
Hello folks, I'd like to start a discussion on adding dynamic routing functionality in Streams sink node. I.e. users do not need to specify the topic name at compilation time but can dynamically determine which topic to send to based on each record's key value pairs. Please find a KIP here: https

[jira] [Resolved] (KAFKA-3473) KIP-237: More Controller Health Metrics

2018-05-14 Thread Dong Lin (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3473?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Dong Lin resolved KAFKA-3473. - Resolution: Fixed > KIP-237: More Controller Health Metrics > --- > >