Re: 1.1 KIPs

2018-01-24 Thread Damian Guy
Hi, The KIP deadline has passed and i've updated the release plan: https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=75957546 If there is anything i've missed please let me know. The feature freeze deadline is January 30th. At this point i'll cut the branch for 1.1. So please make

[jira] [Created] (KAFKA-6480) Add config to enforce max fetch size on the broker

2018-01-24 Thread Jason Gustafson (JIRA)
Jason Gustafson created KAFKA-6480: -- Summary: Add config to enforce max fetch size on the broker Key: KAFKA-6480 URL: https://issues.apache.org/jira/browse/KAFKA-6480 Project: Kafka Issue

Re: [VOTE] KIP-232: Detect outdated metadata using leaderEpoch and partitionEpoch

2018-01-24 Thread Colin McCabe
Hi Dong, Thanks for proposing this KIP. I think a metadata epoch is a really good idea. I read through the DISCUSS thread, but I still don't have a clear picture of why the proposal uses a metadata epoch per partition rather than a global metadata epoch. A metadata epoch per partition is

Re: 1.1 KIPs

2018-01-24 Thread Vahid S Hashemian
Hi Damian, Could you please add KIP-229 to the list? It was approved earlier this week https://www.mail-archive.com/dev@kafka.apache.org/msg84851.html Thanks for running the release. --Vahid From: Damian Guy To: dev@kafka.apache.org Date: 01/24/2018 01:20 PM

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

2018-01-24 Thread Apache Jenkins Server
See -- Started by an SCM change [EnvInject] - Loading node environment variables. Building remotely on ubuntu-eu2 (ubuntu trusty) in workspace

Re: Streams - retry configuration publishing changelog updates?

2018-01-24 Thread Guozhang Wang
Hello Dan, It seems you are hitting a known issue that KIP-91 is trying to fix (it is a general issue of producer itself): https://cwiki.apache.org/confluence/display/KAFKA/KIP-91+Provide+Intuitive+User+Timeouts+in+The+Producer It means that the records has never been sent out before expiring

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

2018-01-24 Thread Apache Jenkins Server
See -- Started by an SCM change [EnvInject] - Loading node environment variables. Building remotely on ubuntu-eu2 (ubuntu trusty) in workspace

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

2018-01-24 Thread Apache Jenkins Server
See -- Started by an SCM change [EnvInject] - Loading node environment variables. Building remotely on ubuntu-eu2 (ubuntu trusty) in workspace

Re: [VOTE] KIP-227: Introduce Fetch Requests that are Incremental to Increase Partition Scalability

2018-01-24 Thread Jun Rao
Since this is a server side metric, it's probably better to use Yammer Rate (which has count) for consistency. Thanks, Jun On Tue, Jan 23, 2018 at 10:17 PM, Colin McCabe wrote: > On Tue, Jan 23, 2018, at 21:47, Ismael Juma wrote: > > Colin, > > > > You get a cumulative

[jira] [Created] (KAFKA-6481) Improving performance of the function ControllerChannelManager.addUpdateMetadataRequestForBrokers

2018-01-24 Thread Lucas Wang (JIRA)
Lucas Wang created KAFKA-6481: - Summary: Improving performance of the function ControllerChannelManager.addUpdateMetadataRequestForBrokers Key: KAFKA-6481 URL: https://issues.apache.org/jira/browse/KAFKA-6481

[DISCUSS] Improving ACLs by allowing ip ranges and subnet expressions?

2018-01-24 Thread Sönke Liebau
Hi everyone, the current ACL functionality in Kafka is a bit limited concerning host based rules when specifying multiple hosts. A common scenario for this would be that if have a YARN cluster running Spark jobs that access Kafka and want to create ACLs based on the ip addresses of the cluster

Re: [DISCUSS] Improving ACLs by allowing ip ranges and subnet expressions?

2018-01-24 Thread Colin McCabe
Hi Sonke, IP address based security doesn't really work, though. Users can spoof IP addresses. They can poison the ARP cache on a local network, or impersonate a DNS server. For users who want some access controls, but don't care about security, maybe we should make it easier to use and

Re: [DISCUSS] KIP-250 Add Support for Quorum-based Producer Acknowledgment

2018-01-24 Thread Litao Deng
Sorry folks, just realized I didn't use the correct thread format for the discussion. I started this new one and copied all of the responses from the old one. @Dong It makes sense to just use the min.insync.replicas instead of introducing a new config, and we must make this change together with

Re: [DISCUSS] Improving ACLs by allowing ip ranges and subnet expressions?

2018-01-24 Thread Sönke Liebau
Hi Colin, I agree with you on the fact that IP based security is not absolute. I was considering it as an additional layer of security to be used in conjunction with ssl certificates, so the rule would contain both the principal and some hosts. This way if someone manages to obtain the

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

2018-01-24 Thread Apache Jenkins Server
See

Re: [VOTE] KIP-232: Detect outdated metadata using leaderEpoch and partitionEpoch

2018-01-24 Thread Dong Lin
Hey Colin, Thanks for reviewing the KIP. If I understand you right, you maybe suggesting that we can use a global metadataEpoch that is incremented every time controller updates metadata. The problem with this solution is that, if a topic is deleted and created again, user will not know whether

Re: [VOTE] KIP-227: Introduce Fetch Requests that are Incremental to Increase Partition Scalability

2018-01-24 Thread Ismael Juma
Agreed, Jun. Ismael On Wed, Jan 24, 2018 at 4:08 PM, Jun Rao wrote: > Since this is a server side metric, it's probably better to use Yammer Rate > (which has count) for consistency. > > Thanks, > > Jun > > On Tue, Jan 23, 2018 at 10:17 PM, Colin McCabe

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

2018-01-24 Thread Apache Jenkins Server
See Changes: [wangguoz] KAFKA-6462: fix unstable ResetIntegrationTest (#4446) -- [...truncated 1.48 MB...] org.apache.kafka.streams.integration.EosIntegrationTest >

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

2018-01-24 Thread Apache Jenkins Server
See Changes: [wangguoz] KAFKA-6462: fix unstable ResetIntegrationTest (#4446) -- [...truncated 1.87 MB...]

Re: [DISCUSS] KIP-250 Add Support for Quorum-based Producer Acknowledgment

2018-01-24 Thread Litao Deng
Thanks Jun for the detailed feedback. Yes, for #1, I mean the live replicas from the ISR. Actually, I believe for all of the 4 new leader election strategies (offline, reassign, preferred replica and controlled shutdown), we need to make corresponding changes. Will document the details in the

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

2018-01-24 Thread Apache Jenkins Server
See

Re: [DISCUSS] KIP-250 Add Support for Quorum-based Producer Acknowledgment

2018-01-24 Thread Guozhang Wang
Thanks for the KIP Litao. 1. I agree with Dong that it would be better to reuse on the existing config if possible, and with that regards I also agree with Jun's point #2 that previously, there is a confusion on the min.isr's semantics from the user's perspective, and we learned that it is

Re: [VOTE] KIP-232: Detect outdated metadata using leaderEpoch and partitionEpoch

2018-01-24 Thread Guozhang Wang
Yeah that makes sense, again I'm just making sure we understand all the scenarios and what to expect. I agree that if, more generally speaking, say users have only consumed to offset 8, and then call seek(16) to "jump" to a further position, then she needs to be aware that OORE maybe thrown and

Re: [VOTE] KIP-232: Detect outdated metadata using leaderEpoch and partitionEpoch

2018-01-24 Thread Dong Lin
Thanks much for reviewing the KIP! Dong On Wed, Jan 24, 2018 at 7:10 AM, Guozhang Wang wrote: > Yeah that makes sense, again I'm just making sure we understand all the > scenarios and what to expect. > > I agree that if, more generally speaking, say users have only consumed

Re: [DISCUSS] KIP 226 - Dynamic Broker Configuration

2018-01-24 Thread Rajini Sivaram
Hi Ismael, Yes, that makes sense. Looking at the command line options for different tools, we seem to be using *--command-config *in the commands that currently talk to the new AdminClient (DelegationTokenCommand, ConsumerGroupCommand, DeleteRecordsCommand). So perhaps it makes sense to do the

[jira] [Created] (KAFKA-6479) Broker file descriptor leak after consumer request timeout

2018-01-24 Thread Ryan Leslie (JIRA)
Ryan Leslie created KAFKA-6479: -- Summary: Broker file descriptor leak after consumer request timeout Key: KAFKA-6479 URL: https://issues.apache.org/jira/browse/KAFKA-6479 Project: Kafka Issue

Re: [VOTE] KIP-232: Detect outdated metadata using leaderEpoch and partitionEpoch

2018-01-24 Thread Dong Lin
Yes, in general we can not prevent OffsetOutOfRangeException if user seeks to a wrong offset. The main goal is to prevent OffsetOutOfRangeException if user has done things in the right way, e.g. user should know that there is message with this offset. For example, if user calls seek(..) right

Re: [VOTE] KIP-249: Add Delegation Token Operations to Kafka Admin Client

2018-01-24 Thread Satish Duggana
+1, thanks for the KIP. ~Satish. On Wed, Jan 24, 2018 at 5:09 AM, Jun Rao wrote: > Hi, Mani, > > Thanks for the KIP. +1 > > Jun > > On Sun, Jan 21, 2018 at 7:44 AM, Manikumar > wrote: > > > Hi All, > > > > I would like to start a vote on KIP-249

[jira] [Created] (KAFKA-6478) kafka-run-class.bat fails if CLASSPATH contains spaces

2018-01-24 Thread Bert Roos (JIRA)
Bert Roos created KAFKA-6478: Summary: kafka-run-class.bat fails if CLASSPATH contains spaces Key: KAFKA-6478 URL: https://issues.apache.org/jira/browse/KAFKA-6478 Project: Kafka Issue Type: Bug

Re: [VOTE] KIP-86: Configurable SASL callback handlers

2018-01-24 Thread Manikumar
Hi, +1 (non-binding) Thanks for the KIP. On Wed, Jan 24, 2018 at 5:00 AM, Jun Rao wrote: > Hi, Rajini, > > Thanks for the KIP. +1 from me. > > Jun > > On Thu, Jan 18, 2018 at 8:58 AM, tao xiao wrote: > > > +1 (non-binding) > > > > On Fri, 19 Jan 2018

Re: offsetsForTimes API performance

2018-01-24 Thread srimugunthan dhandapani
Does the performance of kafka APIs ( https://kafka.apache.org/0110/javadoc/org/apache/kafka/clients/consumer/KafkaConsumer.html ) depend on how geographically apart the caller of the API is from the kafka cluster? Do all APIs perform faster if the calls are made from a machine co-located in the

Re: [VOTE] KIP-86: Configurable SASL callback handlers

2018-01-24 Thread Ted Yu
+1 Original message From: Manikumar Date: 1/24/18 3:07 AM (GMT-08:00) To: dev@kafka.apache.org Subject: Re: [VOTE] KIP-86: Configurable SASL callback handlers Hi, +1 (non-binding) Thanks for the KIP. On Wed, Jan 24, 2018 at 5:00 AM, Jun Rao

Re: [DISCUSS] KIP 226 - Dynamic Broker Configuration

2018-01-24 Thread Viktor Somogyi
Hi all, I'd also like to as the community here who were participating the discussion of KIP-226 to take a look at KIP-248 (that is making kafka-configs.sh fully function with AdminClient and a Java based ConfigCommand). It would be much appreciated to get feedback on that as it plays an important

Re: [VOTE] KIP-86: Configurable SASL callback handlers

2018-01-24 Thread Ismael Juma
Thanks for the KIP, Rajini. This is a useful improvement, so +1 (binding) from me. I really don't like how the Java Security classes work, so I would have preferred to avoid emulating them, but the KIP is consistent with previous related KIPs and that's the direction we chose previously. Also, I

Re: [VOTE] KIP-86: Configurable SASL callback handlers

2018-01-24 Thread Rajini Sivaram
Thanks everyone for the feedback and votes. The vote has passed with 3 binding votes (Jun, Ismael, me) and 6 non-binding votes (Edo, Mickael, Tom, Ted, Tao, Manikumar). I will update the KIP page. Regards, Rajini On Wed, Jan 24, 2018 at 7:26 AM, Ismael Juma wrote: >