[GitHub] kafka pull request #1722: remove incomplete gradle wrapper infrastructure

2016-08-12 Thread kamilszymanski
GitHub user kamilszymanski opened a pull request: https://github.com/apache/kafka/pull/1722 remove incomplete gradle wrapper infrastructure Since jar files (including gradle-wrapper.jar) have to be excluded from source releases current gradle wrapper infrastructure was incomplete re

[jira] [Commented] (KAFKA-3916) Connection from controller to broker disconnects

2016-08-12 Thread Michal Turek (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3916?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15418477#comment-15418477 ] Michal Turek commented on KAFKA-3916: - Hi Jason and others, I have finally found the

[jira] [Commented] (KAFKA-3937) Kafka Clients Leak Native Memory For Longer Than Needed With Compressed Messages

2016-08-12 Thread William Yu (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3937?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15418276#comment-15418276 ] William Yu commented on KAFKA-3937: --- You're referring to the Kafka consumer when you say

Re: [DISCUSS] KIP-73: Replication Quotas

2016-08-12 Thread Jun Rao
Mayuresh, I was thinking of the following. If P1 has data and P2 is throttled, we will return empty data for P2 and send the response back immediately. The follower will issue the next fetch request immediately, but the leader won't return any data in P2 until the quota is not exceeded. We are no

[jira] [Created] (KAFKA-4035) AclCommand should allow Describe operation on groups

2016-08-12 Thread Jun Rao (JIRA)
Jun Rao created KAFKA-4035: -- Summary: AclCommand should allow Describe operation on groups Key: KAFKA-4035 URL: https://issues.apache.org/jira/browse/KAFKA-4035 Project: Kafka Issue Type: Bug

[GitHub] kafka pull request #1721: KAFKA-3845: KIP-75: Add per-connector converters

2016-08-12 Thread ewencp
GitHub user ewencp opened a pull request: https://github.com/apache/kafka/pull/1721 KAFKA-3845: KIP-75: Add per-connector converters You can merge this pull request into a Git repository by running: $ git pull https://github.com/ewencp/kafka kafka-3845-per-connector-converter

Re: [DISCUSS] KIP-71 Enable log compaction and deletion to co-exist

2016-08-12 Thread Damian Guy
Hi, The only concrete example i can think of is a case for limiting disk usage. Say, i had something like Connect running that was tracking changes in a database. Downstream i don't really care about every change, i just want the latest values, so compaction could be enabled. However, the kafka cl

[jira] [Assigned] (KAFKA-3595) Add capability to specify replication compact option for stream store

2016-08-12 Thread Damian Guy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3595?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Damian Guy reassigned KAFKA-3595: - Assignee: Damian Guy > Add capability to specify replication compact option for stream store > --

[jira] [Assigned] (KAFKA-3452) Support session windows besides time interval windows

2016-08-12 Thread Damian Guy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3452?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Damian Guy reassigned KAFKA-3452: - Assignee: Damian Guy > Support session windows besides time interval windows > --

[jira] [Commented] (KAFKA-3845) Support per-connector converters

2016-08-12 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3845?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15418352#comment-15418352 ] ASF GitHub Bot commented on KAFKA-3845: --- GitHub user ewencp opened a pull request:

[GitHub] kafka pull request #1720: KAFKA-4034: Avoid unnecessary consumer coordinator...

2016-08-12 Thread hachikuji
GitHub user hachikuji opened a pull request: https://github.com/apache/kafka/pull/1720 KAFKA-4034: Avoid unnecessary consumer coordinator lookup You can merge this pull request into a Git repository by running: $ git pull https://github.com/hachikuji/kafka KAFKA-4034 Alternat

Re: [DISCUSS] KIP-74: Add Fetch Response Size Limit in Bytes

2016-08-12 Thread Jun Rao
Hi, Andrey, One potential benefit of keeping the per partition limit is for Kafka stream. When reading messages from different partitions, KStream prefers to read from partitions with smaller timestamps first and only advances the KStream timestamp when it sees at least one message from every part

Re: [DISCUSS] KIP-74: Add Fetch Response Size Limit in Bytes

2016-08-12 Thread Ben Stopford
Andrey To make progress, I suggest you keep the partition-level limit in, at least for now, and keep it on the FetchRequest too. B > On 10 Aug 2016, at 18:55, Andrey L. Neporada wrote: > > Hi, Jun! > > Thanks for feedback! > >> On 10 Aug 2016, at 17:42, Jun Rao wrote: >> >> Hi, Andrey,

Re: [DISCUSS] KIP-73: Replication Quotas

2016-08-12 Thread Ben Stopford
Hi Mayuresh So, we’re not actually delaying the FetchResponse, on the leader, unless it is empty i.e. less min.bytes, ready to be sent, for both throttled and non-thorttled partitions. So there shouldn’t be any need for the leader to ask the follower to delay. If there are bytes to be transmit

[GitHub] kafka pull request #1625: MINOR: fix extra broker setup

2016-08-12 Thread mweirauch
Github user mweirauch closed the pull request at: https://github.com/apache/kafka/pull/1625 --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is e

[jira] [Resolved] (KAFKA-3909) Queryable state for Kafka Streams

2016-08-12 Thread Damian Guy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3909?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Damian Guy resolved KAFKA-3909. --- Resolution: Fixed > Queryable state for Kafka Streams > - > >

Re: Review request for KAFKA-3600

2016-08-12 Thread Gwen Shapira
I am 100% pro smart Java clients that support KIP-35 and can use it to work with newer brokers. If this JIRA makes sense as a step in that direction, I think its great and remove my objection. I didn't see anything that looked like a plan toward full forward-backward compatibility, which is why I

Re: [DISCUSS] KIP-74: Add Fetch Response Size Limit in Bytes

2016-08-12 Thread Andrey L. Neporada
Hi! > On 12 Aug 2016, at 04:29, Jun Rao wrote: > > Hi, Andrey, > > One potential benefit of keeping the per partition limit is for Kafka > stream. When reading messages from different partitions, KStream prefers to > read from partitions with smaller timestamps first and only advances the > KSt

Re: [DISCUSS] KIP-74: Add Fetch Response Size Limit in Bytes

2016-08-12 Thread Andrey L. Neporada
Thanks! Will do. > On 12 Aug 2016, at 08:29, Ben Stopford wrote: > > Andrey > > To make progress, I suggest you keep the partition-level limit in, at least > for now, and keep it on the FetchRequest too. > > B >

Re: [DISCUSS] KIP-74: Add Fetch Response Size Limit in Bytes

2016-08-12 Thread Ismael Juma
That's a good point Jun and I agree that it makes sense to keep both limits in that case. Ismael On Fri, Aug 12, 2016 at 2:29 AM, Jun Rao wrote: > Hi, Andrey, > > One potential benefit of keeping the per partition limit is for Kafka > stream. When reading messages from different partitions, KSt

[jira] [Commented] (KAFKA-3937) Kafka Clients Leak Native Memory For Longer Than Needed With Compressed Messages

2016-08-12 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3937?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15418678#comment-15418678 ] Ismael Juma commented on KAFKA-3937: [~williamyu], is your topic compressed? > Kafka

[GitHub] kafka pull request #1718: KAFKA-4031: Check if buffer cleaner is null before...

2016-08-12 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/1718 --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enab

[jira] [Commented] (KAFKA-4031) Check DirectBuffer's cleaner to be not null before using

2016-08-12 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4031?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15418685#comment-15418685 ] ASF GitHub Bot commented on KAFKA-4031: --- Github user asfgit closed the pull request

[jira] [Updated] (KAFKA-4031) Check DirectBuffer's cleaner to be not null before using

2016-08-12 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4031?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-4031: --- Fix Version/s: (was: 0.10.0.2) 0.10.1.0 > Check DirectBuffer's cleaner to be no

[jira] [Resolved] (KAFKA-4031) Check DirectBuffer's cleaner to be not null before using

2016-08-12 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4031?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma resolved KAFKA-4031. Resolution: Fixed Reviewer: Ismael Juma > Check DirectBuffer's cleaner to be not null before u

[jira] [Commented] (KAFKA-3959) __consumer_offsets wrong number of replicas at startup

2016-08-12 Thread Manikumar Reddy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3959?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15418751#comment-15418751 ] Manikumar Reddy commented on KAFKA-3959: looks like this is done on purpose. Whil

[jira] [Comment Edited] (KAFKA-3937) Kafka Clients Leak Native Memory For Longer Than Needed With Compressed Messages

2016-08-12 Thread William Yu (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3937?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15418766#comment-15418766 ] William Yu edited comment on KAFKA-3937 at 8/12/16 12:43 PM: -

[jira] [Commented] (KAFKA-3937) Kafka Clients Leak Native Memory For Longer Than Needed With Compressed Messages

2016-08-12 Thread William Yu (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3937?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15418766#comment-15418766 ] William Yu commented on KAFKA-3937: --- [~ijuma], yes the topic is compressed, are producer

[jira] [Commented] (KAFKA-3959) __consumer_offsets wrong number of replicas at startup

2016-08-12 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3959?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15418819#comment-15418819 ] Ismael Juma commented on KAFKA-3959: The current behaviour "minimum of (available brok

[jira] [Commented] (KAFKA-3937) Kafka Clients Leak Native Memory For Longer Than Needed With Compressed Messages

2016-08-12 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3937?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15418824#comment-15418824 ] Ismael Juma commented on KAFKA-3937: Seems like this one then. It's the first time we'

[jira] [Comment Edited] (KAFKA-3959) __consumer_offsets wrong number of replicas at startup

2016-08-12 Thread Manikumar Reddy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3959?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15418751#comment-15418751 ] Manikumar Reddy edited comment on KAFKA-3959 at 8/12/16 1:27 PM: ---

[jira] [Assigned] (KAFKA-4035) AclCommand should allow Describe operation on groups

2016-08-12 Thread Manikumar Reddy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4035?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar Reddy reassigned KAFKA-4035: -- Assignee: Manikumar Reddy > AclCommand should allow Describe operation on groups >

[jira] [Commented] (KAFKA-3937) Kafka Clients Leak Native Memory For Longer Than Needed With Compressed Messages

2016-08-12 Thread William Yu (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3937?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15418836#comment-15418836 ] William Yu commented on KAFKA-3937: --- FYI: My brokers and client are both on kafka: 0.9.0

Re: [DISCUSS] KIP-71 Enable log compaction and deletion to co-exist

2016-08-12 Thread Damian Guy
Hi Jason, It is still not clear to me how adding a callback would help. You could remove the periodic task, but you would still need to scan the entire cache to remove the expired offsets. Am i missing something? Thanks, Damian On 11 August 2016 at 10:00, Damian Guy wrote: > Hi Jason, > > Tha

[GitHub] kafka pull request #1723: KAFKA-4035: AclCommand should allow Describe opera...

2016-08-12 Thread omkreddy
GitHub user omkreddy opened a pull request: https://github.com/apache/kafka/pull/1723 KAFKA-4035: AclCommand should allow Describe operation on groups You can merge this pull request into a Git repository by running: $ git pull https://github.com/omkreddy/kafka KAFKA-4035 Alt

[jira] [Commented] (KAFKA-4035) AclCommand should allow Describe operation on groups

2016-08-12 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4035?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15418906#comment-15418906 ] ASF GitHub Bot commented on KAFKA-4035: --- GitHub user omkreddy opened a pull request:

[jira] [Updated] (KAFKA-4035) AclCommand should allow Describe operation on groups

2016-08-12 Thread Manikumar Reddy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4035?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar Reddy updated KAFKA-4035: --- Status: Patch Available (was: Open) > AclCommand should allow Describe operation on groups >

[jira] [Updated] (KAFKA-4035) AclCommand should allow Describe operation on groups

2016-08-12 Thread Manikumar Reddy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4035?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar Reddy updated KAFKA-4035: --- Affects Version/s: 0.9.0.0 0.10.0.0 0.10.0.1 > A

Re: Review request for KAFKA-3600

2016-08-12 Thread Ashish Singh
Most of the work in the patch is to enhance NetworkClient to maintain api versions supported by brokers it has connections to. When a broker disconnects, its api versions info is removed and when it reconnects that info is fetched again. In short, with these changes Network Client at any given poin

[jira] [Commented] (KAFKA-2629) Enable getting SSL password from an executable rather than passing plaintext password

2016-08-12 Thread Ashish K Singh (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2629?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15418941#comment-15418941 ] Ashish K Singh commented on KAFKA-2629: --- [~bharatviswa] the way we do it is that for

[jira] [Commented] (KAFKA-2629) Enable getting SSL password from an executable rather than passing plaintext password

2016-08-12 Thread Ashish K Singh (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2629?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15418943#comment-15418943 ] Ashish K Singh commented on KAFKA-2629: --- [~harsha_ch] any thoughts? > Enable gettin

[jira] [Updated] (KAFKA-4012) KerberosShortNamer should implement toString()

2016-08-12 Thread Manikumar Reddy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4012?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar Reddy updated KAFKA-4012: --- Status: Patch Available (was: Open) > KerberosShortNamer should implement toString() > --

[GitHub] kafka pull request #1666: [KAFKA-3991]: allow MirrorMaker to have custom pro...

2016-08-12 Thread hsun-cnnxty
Github user hsun-cnnxty closed the pull request at: https://github.com/apache/kafka/pull/1666 --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is

[jira] [Commented] (KAFKA-3991) MirrorMaker: allow custom publisher

2016-08-12 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3991?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15418965#comment-15418965 ] ASF GitHub Bot commented on KAFKA-3991: --- Github user hsun-cnnxty closed the pull req

[GitHub] kafka pull request #1666: [KAFKA-3991]: allow MirrorMaker to have custom pro...

2016-08-12 Thread hsun-cnnxty
GitHub user hsun-cnnxty reopened a pull request: https://github.com/apache/kafka/pull/1666 [KAFKA-3991]: allow MirrorMaker to have custom producer Please see jira: https://issues.apache.org/jira/browse/KAFKA-3991 You can merge this pull request into a Git repository by running:

[jira] [Commented] (KAFKA-3991) MirrorMaker: allow custom publisher

2016-08-12 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3991?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15418966#comment-15418966 ] ASF GitHub Bot commented on KAFKA-3991: --- GitHub user hsun-cnnxty reopened a pull req

Would like to start contributing to Kafka project

2016-08-12 Thread shashank Jain
Hi, Can you add me as a contributor to the Kafka project. Thanks Shashank

Re: [DISCUSS] KIP-74: Add Fetch Response Size Limit in Bytes

2016-08-12 Thread Jun Rao
Hi, Andrey, Why shouldn't the client library do reordering? It seems that if ReplicaFetcher thread does round-robin, the consumer client should do that too? Thanks, Jun On Fri, Aug 12, 2016 at 3:56 AM, Andrey L. Neporada < anepor...@yandex-team.ru> wrote: > Hi! > > > On 12 Aug 2016, at 04:29,

Re: [DISCUSS] KIP-71 Enable log compaction and deletion to co-exist

2016-08-12 Thread Jun Rao
Hi, Damian, I was just wondering if we should disable size-based retention in the compact_and_delete mode. So, it sounds like that there could be a use case for that. Since by default, the size-based retention is infinite, I think we can just leave the KIP as it is. Thanks, Jun On Fri, Aug 12,

[GitHub] kafka pull request #1724: [KAFKA-3997] log partition name on truncation

2016-08-12 Thread resetius
GitHub user resetius opened a pull request: https://github.com/apache/kafka/pull/1724 [KAFKA-3997] log partition name on truncation You can merge this pull request into a Git repository by running: $ git pull https://github.com/resetius/kafka KAFKA-3997 Alternatively you can

[jira] [Commented] (KAFKA-3997) Halting because log truncation is not allowed and suspicious logging

2016-08-12 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3997?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15419025#comment-15419025 ] ASF GitHub Bot commented on KAFKA-3997: --- GitHub user resetius opened a pull request:

Re: [DISCUSS] KIP-71 Enable log compaction and deletion to co-exist

2016-08-12 Thread Damian Guy
Thanks Jun On Fri, 12 Aug 2016 at 16:41 Jun Rao wrote: > Hi, Damian, > > I was just wondering if we should disable size-based retention in the > compact_and_delete mode. So, it sounds like that there could be a use case > for that. Since by default, the size-based retention is infinite, I think

How to get pull request reviewed - Pull request # 1539

2016-08-12 Thread Jayesh Thakrar
Hi All, I am new to the Kafka community group development and was wondering how do I get my pull request reviewed/processed? https://github.com/apache/kafka/pull/1539 Am I missing any steps or is it just the backlog? Thanks,Jayesh

[jira] [Commented] (KAFKA-3959) __consumer_offsets wrong number of replicas at startup

2016-08-12 Thread Onur Karaman (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3959?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15419057#comment-15419057 ] Onur Karaman commented on KAFKA-3959: - Perfect timing! [~toddpalino] and I hit this be

[jira] [Commented] (KAFKA-3959) __consumer_offsets wrong number of replicas at startup

2016-08-12 Thread James Cheng (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3959?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15419066#comment-15419066 ] James Cheng commented on KAFKA-3959: You mentioned {bq} but __consumer_offsets append

[jira] [Comment Edited] (KAFKA-3959) __consumer_offsets wrong number of replicas at startup

2016-08-12 Thread James Cheng (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3959?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15419066#comment-15419066 ] James Cheng edited comment on KAFKA-3959 at 8/12/16 4:12 PM: -

Re: [DISCUSS] KIP-74: Add Fetch Response Size Limit in Bytes

2016-08-12 Thread Andrey L. Neporada
Hi! > On 12 Aug 2016, at 18:32, Jun Rao wrote: > > Hi, Andrey, > > Why shouldn't the client library do reordering? It seems that if > ReplicaFetcher thread does round-robin, the consumer client should do that > too? > IMHO the client library is not a good place to implement such logic. For ex

[jira] [Commented] (KAFKA-3545) Generalized Serdes for List/Map

2016-08-12 Thread Dru P (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3545?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15419116#comment-15419116 ] Dru P commented on KAFKA-3545: -- [~gfodor] and [~guozhang] - How can I contribute to this proj

[jira] [Commented] (KAFKA-3959) __consumer_offsets wrong number of replicas at startup

2016-08-12 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3959?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15419108#comment-15419108 ] Ismael Juma commented on KAFKA-3959: [~onurkaraman], I would agree. [~guozhang] propos

[jira] [Commented] (KAFKA-3959) __consumer_offsets wrong number of replicas at startup

2016-08-12 Thread Onur Karaman (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3959?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15419124#comment-15419124 ] Onur Karaman commented on KAFKA-3959: - Another options is to offload the responsibilit

Re: Review request for KAFKA-3600

2016-08-12 Thread Dana Powers
I think one of the hard parts is that while we agree that a call to ApiVersions is required, I don't think there is agreement on how to use the response. In kafka-python, for example, to support backwards compatibility we set a single "api" configuration value during the client constructor. This v

[jira] [Commented] (KAFKA-2629) Enable getting SSL password from an executable rather than passing plaintext password

2016-08-12 Thread Sriharsha Chintalapani (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2629?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15419134#comment-15419134 ] Sriharsha Chintalapani commented on KAFKA-2629: --- [~singhashish] happy to rev

[jira] [Commented] (KAFKA-3959) __consumer_offsets wrong number of replicas at startup

2016-08-12 Thread Manikumar Reddy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3959?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15419140#comment-15419140 ] Manikumar Reddy commented on KAFKA-3959: Yes, min.insync.replicas applicable to

Re: [DISCUSS] KIP-74: Add Fetch Response Size Limit in Bytes

2016-08-12 Thread Jun Rao
Hi, Andrey, Yes, I agree that it's more work for the client to do the round-robin logic since it has to be stateful. However, that applies to both the consumer client and the replica fetch thread. I just feel that it's weird to use one strategy in the replica fetch thread and another in the consum

[jira] [Commented] (KAFKA-3408) consumer rebalance fail

2016-08-12 Thread Dru P (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3408?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15419157#comment-15419157 ] Dru P commented on KAFKA-3408: -- [~sdhzlzhk] The information you've provided is insufficient t

Re: [DISCUSS] Time-based releases for Apache Kafka

2016-08-12 Thread Nacho Solis
​How would time releases relate to versions? (Major, minor, API compatibility, etc).​ On Thu, Aug 11, 2016 at 9:37 AM, Guozhang Wang wrote: > I think we do not need to make the same guarantee as for "how old of your > Kafka version that you can upgrade to the latest in one shot" (just call it >

[GitHub] kafka pull request #1725: WIP KAFKA-3894: split log segment to avoid crashin...

2016-08-12 Thread tcrayford
GitHub user tcrayford opened a pull request: https://github.com/apache/kafka/pull/1725 WIP KAFKA-3894: split log segment to avoid crashing cleaner thread https://issues.apache.org/jira/browse/KAFKA-3894 This is a temporary PR, to see what Jenkins has to say about this work i

[jira] [Commented] (KAFKA-3894) LogCleaner thread crashes if not even one segment can fit in the offset map

2016-08-12 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3894?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15419166#comment-15419166 ] ASF GitHub Bot commented on KAFKA-3894: --- GitHub user tcrayford opened a pull request

Re: [DISCUSS] KIP-74: Add Fetch Response Size Limit in Bytes

2016-08-12 Thread Andrey L. Neporada
Hi! > On 12 Aug 2016, at 20:22, Jun Rao wrote: > > Hi, Andrey, > > Yes, I agree that it's more work for the client to do the round-robin logic > since it has to be stateful. However, that applies to both the consumer > client and the replica fetch thread. I just feel that it's weird to use one >

[GitHub] kafka pull request #1726: WIP: KAFKA-4033: KIP-70: Revise Partition Assignme...

2016-08-12 Thread vahidhashemian
GitHub user vahidhashemian opened a pull request: https://github.com/apache/kafka/pull/1726 WIP: KAFKA-4033: KIP-70: Revise Partition Assignment Semantics on New Consumer's Subscription Change You can merge this pull request into a Git repository by running: $ git pull https:

[jira] [Commented] (KAFKA-4033) KIP-70: Revise Partition Assignment Semantics on New Consumer's Subscription Change

2016-08-12 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4033?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15419250#comment-15419250 ] ASF GitHub Bot commented on KAFKA-4033: --- GitHub user vahidhashemian opened a pull re

[GitHub] kafka pull request #1727: KAFKA-3847: Use a separate producer per source tas...

2016-08-12 Thread ewencp
GitHub user ewencp opened a pull request: https://github.com/apache/kafka/pull/1727 KAFKA-3847: Use a separate producer per source task You can merge this pull request into a Git repository by running: $ git pull https://github.com/ewencp/kafka kafka-3847-per-task-producers A

[jira] [Updated] (KAFKA-3847) Connect tasks should not share a producer

2016-08-12 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3847?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava updated KAFKA-3847: - Status: Patch Available (was: Open) > Connect tasks should not share a producer >

[jira] [Commented] (KAFKA-3847) Connect tasks should not share a producer

2016-08-12 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3847?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15419261#comment-15419261 ] ASF GitHub Bot commented on KAFKA-3847: --- GitHub user ewencp opened a pull request:

Re: [DISCUSS] KIP-71 Enable log compaction and deletion to co-exist

2016-08-12 Thread Jason Gustafson
Hey Damian, That's true, but it would avoid the need to write tombstones for the expired offsets I guess. I'm actually not sure it's a great idea anyway. One thing we've talked about is not expiring any offsets as long as a group is alive, which would require some custom expiration logic. There's

[jira] [Commented] (KAFKA-4032) Uncaught exceptions when autocreating topics

2016-08-12 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4032?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15419348#comment-15419348 ] Ismael Juma commented on KAFKA-4032: cc [~granthenke] > Uncaught exceptions when auto

[jira] [Assigned] (KAFKA-4032) Uncaught exceptions when autocreating topics

2016-08-12 Thread Grant Henke (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4032?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Grant Henke reassigned KAFKA-4032: -- Assignee: Grant Henke > Uncaught exceptions when autocreating topics >

[jira] [Commented] (KAFKA-4032) Uncaught exceptions when autocreating topics

2016-08-12 Thread Grant Henke (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4032?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15419350#comment-15419350 ] Grant Henke commented on KAFKA-4032: I will make a patch for this shortly > Uncaught

[DISCUSS] KIP-75 - Add per-connector Converters

2016-08-12 Thread Ewen Cheslack-Postava
Hi all, I've added KIP-75 - Add per-connector Converters https://cwiki.apache.org/confluence/display/KAFKA/KIP-75+-+Add+per-connector+Converters This is a very small completely backwards compatible change, a patch is already available https://github.com/apache/kafka/pull/1721, and I think probab

[jira] [Commented] (KAFKA-3959) __consumer_offsets wrong number of replicas at startup

2016-08-12 Thread Todd Palino (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3959?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15419365#comment-15419365 ] Todd Palino commented on KAFKA-3959: Agree with Onur 100% here. We've been running int

Re: [DISCUSS] KIP-75 - Add per-connector Converters

2016-08-12 Thread Ismael Juma
Sounds sensible to me. Ismael On Fri, Aug 12, 2016 at 8:56 PM, Ewen Cheslack-Postava wrote: > Hi all, > > I've added KIP-75 - Add per-connector Converters > > https://cwiki.apache.org/confluence/display/KAFKA/KIP- > 75+-+Add+per-connector+Converters > > This is a very small completely backwards

Re: [DISCUSS] Time-based releases for Apache Kafka

2016-08-12 Thread Gwen Shapira
Good question! My thoughts are... bugfix releases will be done "as needed" based on community feedback Feature releases will be a minor release by default 0.11, 0.12 - unless: 1. We manage to break compatibility completely (we shouldn't) in which case we need to bump to 1.X 2. We do something tot

[jira] [Commented] (KAFKA-3408) consumer rebalance fail

2016-08-12 Thread kambiz shahri (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3408?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15419424#comment-15419424 ] kambiz shahri commented on KAFKA-3408: -- Yes sure. > consumer rebalance fail > --

Stream processing meetup at LinkedIn (Mountain View) on Tuesday, August 23 at 6pm

2016-08-12 Thread Joel Koshy
Hi everyone, We would like to invite you to a Stream Processing Meetup at LinkedIn’s *Mountain View campus on Tuesday, August 23 at 6pm*. Please RSVP here (only if you intend to attend in person): https://www.meetup.com/Stream-Processing-Meetup-LinkedIn/events/232864129 We have three great talks

[jira] [Commented] (KAFKA-3408) consumer rebalance fail

2016-08-12 Thread Amey (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3408?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15419429#comment-15419429 ] Amey commented on KAFKA-3408: - I also tried but can't reproduce this. Surely we can collaborat

[jira] [Commented] (KAFKA-3959) __consumer_offsets wrong number of replicas at startup

2016-08-12 Thread Onur Karaman (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3959?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15419441#comment-15419441 ] Onur Karaman commented on KAFKA-3959: - btw I can pick this one up once we agree on the

[jira] [Created] (KAFKA-4036) Unsafe dictionary access in VerifiableConsumer service

2016-08-12 Thread Jason Gustafson (JIRA)
Jason Gustafson created KAFKA-4036: -- Summary: Unsafe dictionary access in VerifiableConsumer service Key: KAFKA-4036 URL: https://issues.apache.org/jira/browse/KAFKA-4036 Project: Kafka Issu

[jira] [Commented] (KAFKA-3959) __consumer_offsets wrong number of replicas at startup

2016-08-12 Thread Grant Henke (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3959?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15419471#comment-15419471 ] Grant Henke commented on KAFKA-3959: I would like to present an alternative option. Th

Kafka Mirror maker duplicate issue

2016-08-12 Thread Ghosh, Achintya (Contractor)
Hi there, I created a broker as stand by using Kafka Mirror maker but same messages gets consumed by both Source broker and mirror broker. Ex: I send 1000 messages let's say offset value 1 to 1000 and consumed 500 messages from the source broker. Now my broker goes down and want to read rest

[GitHub] kafka pull request #1727: KAFKA-3847: Use a separate producer per source tas...

2016-08-12 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/1727 --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enab

[jira] [Updated] (KAFKA-3847) Connect tasks should not share a producer

2016-08-12 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3847?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira updated KAFKA-3847: Resolution: Fixed Status: Resolved (was: Patch Available) Issue resolved by pull request 17

[jira] [Commented] (KAFKA-3847) Connect tasks should not share a producer

2016-08-12 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3847?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15419482#comment-15419482 ] ASF GitHub Bot commented on KAFKA-3847: --- Github user asfgit closed the pull request

[jira] [Commented] (KAFKA-3408) consumer rebalance fail

2016-08-12 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3408?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15419489#comment-15419489 ] Ewen Cheslack-Postava commented on KAFKA-3408: -- [~dpnchl] I've added you as a

Re: [DISCUSS] Time-based releases for Apache Kafka

2016-08-12 Thread Jun Rao
Right now, our major releases are really indicated in the second digit, not in the leading 0. So 0.10 is a major, 0.10.1 is a minor and 0.10.0.1 is a bug fix. Thanks, Jun On Fri, Aug 12, 2016 at 1:17 PM, Gwen Shapira wrote: > Good question! > > My thoughts are... bugfix releases will be done "

[jira] [Updated] (KAFKA-2946) DeleteTopic - protocol and server side implementation

2016-08-12 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2946?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao updated KAFKA-2946: --- Resolution: Fixed Fix Version/s: 0.10.1.0 Status: Resolved (was: Patch Available) Issue res

[GitHub] kafka pull request #1616: KAFKA-2946: DeleteTopic - protocol and server side...

2016-08-12 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/1616 --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enab

[jira] [Commented] (KAFKA-2946) DeleteTopic - protocol and server side implementation

2016-08-12 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2946?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15419508#comment-15419508 ] ASF GitHub Bot commented on KAFKA-2946: --- Github user asfgit closed the pull request

Re: [DISCUSS] Time-based releases for Apache Kafka

2016-08-12 Thread Ismael Juma
Interesting, so you are suggesting that we drop Kafka's current versioning scheme (0.major.minor.patch)? I can see the reasoning for doing so now. However, I think I'd prefer to do that when we do the 1.x bump, which I think we should do once exactly-once lands. That way people only have to re-lea

Re: [DISCUSS] Time-based releases for Apache Kafka

2016-08-12 Thread Nacho Solis
I'm not convinced that time-based releases for this type of development are the right thing. Something like Ubuntu, where all components are moving targets needs to decide to freeze and release without having full coordination from the participants. There is no guarantee from Canonical that the i

[GitHub] kafka pull request #1728: MINOR: add describe to valid group acl operations

2016-08-12 Thread norwood
GitHub user norwood opened a pull request: https://github.com/apache/kafka/pull/1728 MINOR: add describe to valid group acl operations @junrao You can merge this pull request into a Git repository by running: $ git pull https://github.com/norwood/kafka add-describe-to-valid-g

  1   2   >