Re: [DISCUSS] KIP-252: Extend ACLs to allow filtering based on ip ranges and subnets

2019-04-04 Thread Sönke Liebau
Hi Colin, I agree, we need to have a way of failing incorrect ranges server-side, I'll amend the KIP and look into that. I think INVALID_REQUEST should fit fine, afaik we can send a message along with that code, so that could explain the actual reason. Regarding prohibiting these ACLs from being

Kafka Broker Config (logs.dir) on Kubernetes

2019-04-04 Thread M. Manna
Hello, The question might trigger people to reply with "Confluent" - but it's not related to confluent as the kubernetes offering is not for publi/community edition. So, discussing Helm charts and intro to Confluent isn't our objective here. What I am trying to understand is how does the log fil

[jira] [Resolved] (KAFKA-6399) Consider reducing "max.poll.interval.ms" default for Kafka Streams

2019-04-04 Thread Bill Bejeck (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-6399?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Bill Bejeck resolved KAFKA-6399. Resolution: Fixed > Consider reducing "max.poll.interval.ms" default for Kafka Streams > --

Re: [DISCUSS] KIP-446: Add changelog topic configuration to KTable suppress

2019-04-04 Thread John Roesler
Hi Maarten, Thanks for the KIP! It looks good to me. It seems appropriate to stick close to the same API presented by Materialized. I did notice one mis-statement in the proposed Javadoc: > * Indicates that a changelog should be created for the suppressed KTable. Actually, the changelog is for t

Need help in setting up security in Kafka systems

2019-04-04 Thread Suman B N
Team, Can anyone help me share the configs to be set to achieve the below security in Kafka systems? - Broker-Broker should be PLAINTEXT(No Authentication and Authorization between brokers) - Zookeeper-Broker should be PLAINTEXT(No Authentication and Authorization between brokers and

Re: [DISCUSS] KIP-411: Add option to make Kafka Connect task client ID values unique

2019-04-04 Thread Randall Hauch
Sounds great. I did make a few minor grammatical edits to the "Proposed Changes" section to avoid the notion that the sink and source tasks create the consumers and producers, respectively. I think it's important to accurately denote that the framework creates the producers and consumers for the t

Re: [DISCUSS] KIP-411: Add option to make Kafka Connect task client ID values unique

2019-04-04 Thread pdavidson
Thanks Randall. You're absolutely right that Worker creates the clients before passing them to the tasks, so I'm very happy with your changes. Paul On Thu, Apr 4, 2019 at 8:02 AM Randall Hauch wrote: > Sounds great. > > I did make a few minor grammatical edits to the "Proposed Changes" section

Re: Need help in setting up security in Kafka systems

2019-04-04 Thread M. Manna
Hi, Have you checked the section on security here? It's got a comprehensive guide https://kafka.apache.org/documentation/#security_sasl To answer your questions briefly - Broker to Broker should be plainTEXT (or SSL if inter-broker security is enabled then broker2brorker works as a client coms)

[jira] [Resolved] (KAFKA-8090) Replace ControlledShutdown request/response with automated protocol

2019-04-04 Thread Manikumar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-8090?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar resolved KAFKA-8090. -- Resolution: Fixed Fix Version/s: 2.3.0 Issue resolved by pull request 6423 [https://github.com/a

Jenkins build is back to normal : kafka-trunk-jdk11 #419

2019-04-04 Thread Apache Jenkins Server
See

[jira] [Created] (KAFKA-8189) Streams should have an option to require names for stateful components

2019-04-04 Thread John Roesler (JIRA)
John Roesler created KAFKA-8189: --- Summary: Streams should have an option to require names for stateful components Key: KAFKA-8189 URL: https://issues.apache.org/jira/browse/KAFKA-8189 Project: Kafka

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

2019-04-04 Thread Apache Jenkins Server
See

Re: [VOTE] KIP-369 Alternative Partitioner to Support "Always Round-Robin" Selection

2019-04-04 Thread M. Manna
Hello, Trying to revive this thread again. Would anyone be interested in having this KiP through Thanks, On Fri, 25 Jan 2019 at 16:44, M. Manna wrote: > Hello, > > I am trying to revive this thread. I only got 1 binding vote so far. > > Please feel free to revisit and comment here. > > Thanks

Re: [DISCUSS] KIP-446: Add changelog topic configuration to KTable suppress

2019-04-04 Thread Maarten Duijn
Thank you for the explanation regarding the internals, I have edited the KIP accordingly and updated the Javadoc. About the possible data loss when altering changelog config, I think we can improve by doing (one of) the following. 1) Add a warning in the comments that clearly states what might h

Build failed in Jenkins: kafka-trunk-jdk11 #420

2019-04-04 Thread Apache Jenkins Server
See Changes: [manikumar] KAFKA-8090: Use automatic RPC generation in ControlledShutdown -- [...truncated 2.35 MB...] org.apache.kafka.connect.converters.ByteArrayConverter

Re: [VOTE] KIP-369 Alternative Partitioner to Support "Always Round-Robin" Selection

2019-04-04 Thread Harsha
Looks like the KIP is passed with 3 binding votes. From Matthias, Bill Bejeck and myself you got 3 binding votes. You can do the full tally of the votes and send out a close of vote thread. Thanks, Harsha On Thu, Apr 4, 2019, at 12:24 PM, M. Manna wrote: > Hello, > > Trying to revive this thre

Re: [VOTE] KIP-392: Allow consumers to fetch from the closest replica

2019-04-04 Thread Jason Gustafson
Hi Jun, I have updated the KIP to remove `replica.selection.policy` from the consumer configuration. Thanks for the suggestion. Best, Jason On Wed, Mar 27, 2019 at 9:46 AM Jason Gustafson wrote: > @Jun > > Re; 200: It's a fair point that it is useful to minimize the client > changes that are n

Re: [VOTE] KIP-392: Allow consumers to fetch from the closest replica

2019-04-04 Thread Gwen Shapira
+1 On Thu, Apr 4, 2019 at 2:26 PM Jason Gustafson wrote: > Hi Jun, > > I have updated the KIP to remove `replica.selection.policy` from the > consumer configuration. Thanks for the suggestion. > > Best, > Jason > > On Wed, Mar 27, 2019 at 9:46 AM Jason Gustafson > wrote: > > > @Jun > > > > Re;

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

2019-04-04 Thread Apache Jenkins Server
See Changes: [manikumar] KAFKA-8090: Use automatic RPC generation in ControlledShutdown -- [...truncated 2.36 MB...] org.apache.kafka.connect.data.DateTest > testToLogica

[jira] [Created] (KAFKA-8190) Keystore update without file change doesn't update SSLContext

2019-04-04 Thread Rajini Sivaram (JIRA)
Rajini Sivaram created KAFKA-8190: - Summary: Keystore update without file change doesn't update SSLContext Key: KAFKA-8190 URL: https://issues.apache.org/jira/browse/KAFKA-8190 Project: Kafka

[jira] [Created] (KAFKA-8191) Add pluggability of KeyManager to generate the broker Private Keys and Certificates

2019-04-04 Thread Sai Sandeep (JIRA)
Sai Sandeep created KAFKA-8191: -- Summary: Add pluggability of KeyManager to generate the broker Private Keys and Certificates Key: KAFKA-8191 URL: https://issues.apache.org/jira/browse/KAFKA-8191 Project

Re: [DISCUSS] KIP-360: Improve handling of unknown producer

2019-04-04 Thread Jason Gustafson
Hi Everyone, Sorry for the long delay on this KIP. I have updated it to include the handling of INVALID_PRODUCER_ID_MAPPING as suggested above. If there are no further comments, I will plan to start a vote early next week. Thanks! Jason On Mon, Mar 25, 2019 at 2:33 PM Adam Bellemare wrote: > A

Re: [VOTE] KIP-369 Alternative Partitioner to Support "Always Round-Robin" Selection

2019-04-04 Thread M. Manna
Thanks Harsha. As per your comments, I have counted 3 binding votes so far. Thanks everyone for your comments and support. I’ll update the kip next morning and do the needful. Regards, On Thu, 4 Apr 2019 at 22:10, Harsha wrote: > Looks like the KIP is passed with 3 binding votes. From Matth

Re: Add Jira permission and wiki permission

2019-04-04 Thread Bill Bejeck
My apologies for the delay, you are all set now on the wiki permissions. Thanks, Bill On Tue, Apr 2, 2019 at 6:23 AM slim ouertani wrote: > Hi, > > The account details are as follows: > Full Name Slim Ouertani > Email ouert...@gmail.com > > > Thanks, > Slim > > > On Mon, Apr 1, 2019 at 4:49 PM

Re: [DISCUSS] KIP-236 Interruptible Partition Reassignment

2019-04-04 Thread Jun Rao
Hi, George, Thanks for the KIP. Sorry for the late reply. A couple of comments below. 40. I agree that it's better to issue an RPC request to the controller for reassignment cancellation. If we do that, it would be useful to decide whether that call blocks on cancellation completion or not. 41.

Re: [DISCUSS] KIP-435: Incremental Partition Reassignment

2019-04-04 Thread Jun Rao
Hi, Viktor, Thanks for the KIP. A couple of comments below. 1. Another potential thing to do reassignment incrementally is to move a batch of partitions at a time, instead of all partitions. This may lead to less data replication since by the time the first batch of partitions have been completel

Re: [VOTE] KIP-392: Allow consumers to fetch from the closest replica

2019-04-04 Thread Jun Rao
Hi, Jason, Thanks for the updated KIP. +1 from me. Jun On Thu, Apr 4, 2019 at 2:26 PM Jason Gustafson wrote: > Hi Jun, > > I have updated the KIP to remove `replica.selection.policy` from the > consumer configuration. Thanks for the suggestion. > > Best, > Jason > > On Wed, Mar 27, 2019 at 9:4

Jenkins build is back to normal : kafka-trunk-jdk11 #421

2019-04-04 Thread Apache Jenkins Server
See

[jira] [Created] (KAFKA-8192) Using Failify for e2e testing of Kafka

2019-04-04 Thread Armin Balalaie (JIRA)
Armin Balalaie created KAFKA-8192: - Summary: Using Failify for e2e testing of Kafka Key: KAFKA-8192 URL: https://issues.apache.org/jira/browse/KAFKA-8192 Project: Kafka Issue Type: Test

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

2019-04-04 Thread Apache Jenkins Server
See Changes: [cshapi] MINOR: fix throttling and status in ConnectionStressWorker -- [...truncated 2.35 MB...] org.apache.kafka.connect.runtime.ConnectMetricsTest > testRec

[jira] [Created] (KAFKA-8193) Flaky Test MetricsIntegrationTest#testStreamMetricOfWindowStore

2019-04-04 Thread Konstantine Karantasis (JIRA)
Konstantine Karantasis created KAFKA-8193: - Summary: Flaky Test MetricsIntegrationTest#testStreamMetricOfWindowStore Key: KAFKA-8193 URL: https://issues.apache.org/jira/browse/KAFKA-8193 Proje

Re: [DISCUSS] KIP-252: Extend ACLs to allow filtering based on ip ranges and subnets

2019-04-04 Thread Colin McCabe
On Thu, Apr 4, 2019, at 01:52, Sönke Liebau wrote: > Hi Colin, > > I agree, we need to have a way of failing incorrect ranges server-side, > I'll amend the KIP and look into that. I think INVALID_REQUEST should fit > fine, afaik we can send a message along with that code, so that could > explain t