Re: [VOTE] KIP-70: Revise Partition Assignment Semantics on New Consumer's Subscription Change

2016-08-10 Thread Manikumar Reddy
+1 (non-binding) On Wed, Aug 10, 2016 at 8:30 AM, Ewen Cheslack-Postava wrote: > +1 (binding), thanks for working on this Vahid. > > @Dana - See https://cwiki.apache.org/confluence/display/KAFKA/Bylaws re: > binding/non-binding, although I now notice that we specify criteria (lazy > majority) on

Re: [DISCUSS] KIP-54 Sticky Partition Assignment Strategy

2016-08-10 Thread Andrew Coates
I'm still very interested in seeing this KIP progress ... On Tue, 2 Aug 2016 at 20:09, Vahid S Hashemian wrote: > I would like to revive this thread and ask for additional feedback on this > KIP. > > There have already been some feedback, mostly in favor, plus some concern > about the value gain

[jira] [Commented] (KAFKA-1588) Offset response does not support two requests for the same topic/partition combo

2016-08-10 Thread Magnus Edenhill (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1588?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15414868#comment-15414868 ] Magnus Edenhill commented on KAFKA-1588: [~guozhang] Why would this fix require a

[jira] [Commented] (KAFKA-1588) Offset response does not support two requests for the same topic/partition combo

2016-08-10 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1588?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15414955#comment-15414955 ] Ismael Juma commented on KAFKA-1588: [~edenhill], you're right that it technically doe

[jira] [Commented] (KAFKA-1588) Offset response does not support two requests for the same topic/partition combo

2016-08-10 Thread Magnus Edenhill (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1588?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15414995#comment-15414995 ] Magnus Edenhill commented on KAFKA-1588: Thanks [~ijuma]. But I dont see how the b

[jira] [Commented] (KAFKA-1588) Offset response does not support two requests for the same topic/partition combo

2016-08-10 Thread Magnus Edenhill (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1588?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15415000#comment-15415000 ] Magnus Edenhill commented on KAFKA-1588: I see now that this behaviour is infact d

[jira] [Resolved] (KAFKA-1612) Consumer offsets auto-commit before processing finishes

2016-08-10 Thread Manikumar Reddy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1612?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar Reddy resolved KAFKA-1612. Resolution: Won't Fix Closing this issue in favor new consumer API. > Consumer offsets auto

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

2016-08-10 Thread Andrey L. Neporada
Hi! > On 09 Aug 2016, at 20:46, Jun Rao wrote: > > Hi, Andrey, > > Thanks for the proposal. It looks good overall. Some minor comments. > > 1. It seems that it's bit weird that fetch.partition.max.bytes is a broker > level configuration while fetch.limit.bytes is a client side configuration. >

[GitHub] kafka pull request #1717: KAFKA-3742: Can't run bin/connect-*.sh with -daemo...

2016-08-10 Thread 1ambda
GitHub user 1ambda opened a pull request: https://github.com/apache/kafka/pull/1717 KAFKA-3742: Can't run bin/connect-*.sh with -daemon flag ## Problem Current connect scripts (`connect-distributed.sh`, `connect-standalone.sh`) do not support `-daemon` flag even if users sp

[jira] [Commented] (KAFKA-3742) Can't run connect-distributed.sh with -daemon flag

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

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

2016-08-10 Thread Ben Stopford
Regarding the fetch.partition.max.bytes, there was some discussion on the Jira around removing this setting completely. It’s probably not the easiest thing for user’s to set, so there is certainly an argument for removing it. This would have the side effect that a catching up broker would fill r

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

2016-08-10 Thread Ismael Juma
Hi Gwen, The proposal sounds good to me. With regards to the cadence, 3 releases a year (every 4 months as you said) sounds reasonable. One thing that I think is very important if we release more often is that users should be able to upgrade directly to the latest release for a reasonable period.

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

2016-08-10 Thread Jun Rao
Hi, Andrey, Thanks for the reply. A couple of more comments inline below. On Wed, Aug 10, 2016 at 3:56 AM, Andrey L. Neporada < anepor...@yandex-team.ru> wrote: > Hi! > > > On 09 Aug 2016, at 20:46, Jun Rao wrote: > > > > Hi, Andrey, > > > > Thanks for the proposal. It looks good overall. Some

Re: [DISCUSS] KIP-73: Replication Quotas

2016-08-10 Thread Ben Stopford
Thanks again for the responses everyone. I’ve removed the the extra fetcher threads from the proposal, switching to the inclusion-based approach. The relevant section is: The follower makes a requests, using the fixed size of replica.fetch.response.max.bytes as per KIP-74

[jira] [Created] (KAFKA-4028) Add Connect cluster ID and expose it in REST API

2016-08-10 Thread Ewen Cheslack-Postava (JIRA)
Ewen Cheslack-Postava created KAFKA-4028: Summary: Add Connect cluster ID and expose it in REST API Key: KAFKA-4028 URL: https://issues.apache.org/jira/browse/KAFKA-4028 Project: Kafka

[jira] [Created] (KAFKA-4029) SSL support for Connect REST API

2016-08-10 Thread Ewen Cheslack-Postava (JIRA)
Ewen Cheslack-Postava created KAFKA-4029: Summary: SSL support for Connect REST API Key: KAFKA-4029 URL: https://issues.apache.org/jira/browse/KAFKA-4029 Project: Kafka Issue Type: Im

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

2016-08-10 Thread Gwen Shapira
I hear what you are saying (enterprises upgrade every 2-years more-or-less). It seems reasonable - this basically means maintaining 10 compatibility tests at any point in time. We will need to be disciplined about maintaining those tests though - or it will get painful. Another thing, hurrying up

Strange behavior when turn the system clock back

2016-08-10 Thread Gabriel Ibarra
Hello guys, I am dealing with an issue when turn the system clock back (either due to NTP or administrator action). I'm using kafka_2.11-0.10.0.0 I follow the next steps. - Start a consumer for TOPIC_NAME with group id GROUP_NAME. It will be owner of all the partitions. - Turn the system clock bac

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

2016-08-10 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 reassigned KAFKA-3847: Assignee: Ewen Cheslack-Postava (was: Liquan Pei) > Connect tasks should n

[jira] [Updated] (KAFKA-3971) Consumers drop from coordinator and cannot reconnect

2016-08-10 Thread Andrew Olson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3971?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Andrew Olson updated KAFKA-3971: Summary: Consumers drop from coordinator and cannot reconnect (was: Consumers drop from coordinator

[jira] [Commented] (KAFKA-3752) Provide a way for KStreams to recover from unclean shutdown

2016-08-10 Thread Greg Fodor (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3752?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15415664#comment-15415664 ] Greg Fodor commented on KAFKA-3752: --- Hey [~guozhang], we're hitting this issue as well d

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

2016-08-10 Thread Andrey L. Neporada
Hi, Jun! Thanks for feedback! > On 10 Aug 2016, at 17:42, Jun Rao wrote: > > Hi, Andrey, > > Thanks for the reply. A couple of more comments inline below. > > On Wed, Aug 10, 2016 at 3:56 AM, Andrey L. Neporada < > anepor...@yandex-team.ru > wrote: > >> >> Y

[jira] [Created] (KAFKA-4030) Update older quickstart documents to clarify which version they relate to

2016-08-10 Thread Todd Snyder (JIRA)
Todd Snyder created KAFKA-4030: -- Summary: Update older quickstart documents to clarify which version they relate to Key: KAFKA-4030 URL: https://issues.apache.org/jira/browse/KAFKA-4030 Project: Kafka

[GitHub] kafka-site pull request #17: Update quickstart guides with a note to refer t...

2016-08-10 Thread cptcanuck
GitHub user cptcanuck opened a pull request: https://github.com/apache/kafka-site/pull/17 Update quickstart guides with a note to refer to the current version … https://issues.apache.org/jira/browse/KAFKA-4030 You can merge this pull request into a Git repository by running:

[jira] [Commented] (KAFKA-4030) Update older quickstart documents to clarify which version they relate to

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

[jira] [Commented] (KAFKA-4030) Update older quickstart documents to clarify which version they relate to

2016-08-10 Thread Todd Snyder (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4030?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15415730#comment-15415730 ] Todd Snyder commented on KAFKA-4030: https://github.com/apache/kafka-site/pull/17 > U

[jira] [Updated] (KAFKA-3993) Console producer drops data

2016-08-10 Thread Roger Hoover (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3993?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Roger Hoover updated KAFKA-3993: Description: The console producer drops data when if the process exits too quickly. I suspect that

[jira] [Commented] (KAFKA-3971) Consumers drop from coordinator and cannot reconnect

2016-08-10 Thread Lei Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3971?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15415770#comment-15415770 ] Lei Wang commented on KAFKA-3971: - tried the assigning manually, it didn't help much, stil

[jira] [Commented] (KAFKA-3993) Console producer drops data

2016-08-10 Thread Vahid Hashemian (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3993?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15415793#comment-15415793 ] Vahid Hashemian commented on KAFKA-3993: [~theduderog] I wonder if this is the sam

[jira] [Commented] (KAFKA-3971) Consumers drop from coordinator and cannot reconnect

2016-08-10 Thread Andrew Olson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3971?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15415806#comment-15415806 ] Andrew Olson commented on KAFKA-3971: - How many partitions per topic do you have? > C

[jira] [Commented] (KAFKA-3971) Consumers drop from coordinator and cannot reconnect

2016-08-10 Thread Lei Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3971?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15415810#comment-15415810 ] Lei Wang commented on KAFKA-3971: - one partition each > Consumers drop from coordinator a

[jira] [Commented] (KAFKA-3993) Console producer drops data

2016-08-10 Thread Roger Hoover (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3993?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15415811#comment-15415811 ] Roger Hoover commented on KAFKA-3993: - Thanks, [~vahid]. Yeah, it looks the same. >

[jira] [Commented] (KAFKA-3971) Consumers drop from coordinator and cannot reconnect

2016-08-10 Thread Andrew Olson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3971?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15415838#comment-15415838 ] Andrew Olson commented on KAFKA-3971: - Even at 1 partition, if a single process is con

[jira] [Commented] (KAFKA-3971) Consumers drop from coordinator and cannot reconnect

2016-08-10 Thread Lei Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3971?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15415865#comment-15415865 ] Lei Wang commented on KAFKA-3971: - we don't see too many threads, less than 1k threads at

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

2016-08-10 Thread Soumyajit Sahu (JIRA)
Soumyajit Sahu created KAFKA-4031: - Summary: Check DirectBuffer's cleaner to be not null before using Key: KAFKA-4031 URL: https://issues.apache.org/jira/browse/KAFKA-4031 Project: Kafka Issu

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

2016-08-10 Thread Soumyajit Sahu (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4031?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Soumyajit Sahu reassigned KAFKA-4031: - Assignee: Soumyajit Sahu > Check DirectBuffer's cleaner to be not null before using > ---

RE: Kafka consumer getting duplicate message

2016-08-10 Thread Ghosh, Achintya (Contractor)
Can anyone please check this one? Thanks Achintya -Original Message- From: Ghosh, Achintya (Contractor) Sent: Monday, August 08, 2016 9:44 AM To: us...@kafka.apache.org Cc: dev@kafka.apache.org Subject: RE: Kafka consumer getting duplicate message Thank you , Ewen for your response. Act

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

2016-08-10 Thread soumyajit-sahu
GitHub user soumyajit-sahu opened a pull request: https://github.com/apache/kafka/pull/1718 KAFKA-4031: Check if buffer cleaner is null before using it A small fix to check null before using the reference You can merge this pull request into a Git repository by running: $ git p

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

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

[GitHub] kafka-site issue #17: Update quickstart guides with a note to refer to the c...

2016-08-10 Thread guozhangwang
Github user guozhangwang commented on the issue: https://github.com/apache/kafka-site/pull/17 LGTM. cc @gwenshap --- 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 wish

[jira] [Resolved] (KAFKA-3914) Global discovery of state stores

2016-08-10 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3914?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang resolved KAFKA-3914. -- Resolution: Fixed Issue resolved by pull request 1576 [https://github.com/apache/kafka/pull/1576

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

2016-08-10 Thread Ismael Juma
Hi Gwen, Comments inline. On Wed, Aug 10, 2016 at 6:21 PM, Gwen Shapira wrote: > I hear what you are saying (enterprises upgrade every 2-years > more-or-less). It seems reasonable - this basically means maintaining > 10 compatibility tests at any point in time. Indeed. Although it's up to 6 t

[GitHub] kafka pull request #1576: KAFKA-3914: Global discovery of state stores

2016-08-10 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/1576 --- 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-3914) Global discovery of state stores

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

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

2016-08-10 Thread Dru P (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3408?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15416065#comment-15416065 ] Dru P commented on KAFKA-3408: -- [~ewencp] I'd like to be a contributor for this. [~ajak6] an

[GitHub] kafka pull request #1702: Kafka 3940: Log should check the return value of d...

2016-08-10 Thread imandhan
Github user imandhan closed the pull request at: https://github.com/apache/kafka/pull/1702 --- 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 en

[ANNOUCE] Apache Kafka 0.10.0.1 Released

2016-08-10 Thread Ismael Juma
The Apache Kafka community is pleased to announce the release for Apache Kafka 0.10.0.1. This is a bug fix release that fixes 53 issues in 0.10.0.0. All of the changes in this release can be found in the release notes: *https://archive.apache.org/dist/kafka/0.10.0.1/RELEASE_NOTES.html

[ANNOUCE] Apache Kafka 0.10.0.1 Released

2016-08-10 Thread Ismael Juma
The Apache Kafka community is pleased to announce the release for Apache Kafka 0.10.0.1. This is a bug fix release that fixes 53 issues in 0.10.0.0. All of the changes in this release can be found in the release notes: *https://archive.apache.org/dist/kafka/0.10.0.1/RELEASE_NOTES.html

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

2016-08-10 Thread Jason Gustafson
Hi Damian, Thanks for the KIP. We have a number of use cases in which we maintain a materialized cache of a compacted topic. The consumer coordinator, for example, has a cache of consumer offsets which is populated from the __consumer_offsets topic. Kafka Connect also uses this pattern for its own

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

2016-08-10 Thread Joel Koshy
On Tue, Aug 9, 2016 at 4:49 PM, Gwen Shapira wrote: > > 4. Frequent releases mean we need to do bugfix releases for older > branches. Right now we only do bugfix releases to latest release. > I'm a bit unclear on how the above is a side-effect of time-based releases. IIUC this just changes how f

[jira] [Commented] (KAFKA-3752) Provide a way for KStreams to recover from unclean shutdown

2016-08-10 Thread Greg Fodor (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3752?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15416351#comment-15416351 ] Greg Fodor commented on KAFKA-3752: --- Oh apologies for mis-reading the ticket, but in our

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

2016-08-10 Thread Joel Koshy
On Wed, Aug 10, 2016 at 5:44 PM, Joel Koshy wrote: > > > On Tue, Aug 9, 2016 at 4:49 PM, Gwen Shapira wrote: > >> >> 4. Frequent releases mean we need to do bugfix releases for older >> branches. Right now we only do bugfix releases to latest release. >> > > I'm a bit unclear on how the above is

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

2016-08-10 Thread Gwen Shapira
Good question, let me clarify my thinking: We were used to doing every year (or even at lower frequency). So the expectation was that users will just upgrade once a year and we wouldn't worry about backporting bugfixes to bugs that were over a year old. It seemed pretty reasonable. But if we are

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

2016-08-10 Thread Gwen Shapira
Yeah, I agree that maintaining 6 release branches is not really sustainable... Maybe 3 (current and 2 older) makes sense? On Wed, Aug 10, 2016 at 7:35 PM, Joel Koshy wrote: > On Wed, Aug 10, 2016 at 5:44 PM, Joel Koshy wrote: > >> >> >> On Tue, Aug 9, 2016 at 4:49 PM, Gwen Shapira wrote: >> >>

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

2016-08-10 Thread Jason Gustafson (JIRA)
Jason Gustafson created KAFKA-4032: -- Summary: Uncaught exceptions when autocreating topics Key: KAFKA-4032 URL: https://issues.apache.org/jira/browse/KAFKA-4032 Project: Kafka Issue Type: Bu

Re: [ANNOUCE] Apache Kafka 0.10.0.1 Released

2016-08-10 Thread Gwen Shapira
Woohoo! Thank you, Ismael! You make a great release manager :) On Wed, Aug 10, 2016 at 5:01 PM, Ismael Juma wrote: > The Apache Kafka community is pleased to announce the release for Apache > Kafka 0.10.0.1. > This is a bug fix release that fixes 53 issues in 0.10.0.0. > > All of the changes in

Re: [DISCUSS] KIP-73: Replication Quotas

2016-08-10 Thread Mayuresh Gharat
This might have been answered before. I was wondering when the leader quota is reached and it sends empty response ( If the inclusion of a partition, listed in the leader's throttled-replicas list, causes the LeaderQuotaRate to be exceeded, that partition is omitted from the response (aka returns 0

Re: Review request for KAFKA-3600

2016-08-10 Thread Gwen Shapira
I hate doing this, because Ashish has really been good about following up on the PR, but I'm questioning the usefulness of this patch. It adds non-trivial complexity to the client... with not much return on the investment, as far as I can see? When I first suggested it, it was before KIP-35 was me

Re: Strange behavior when turn the system clock back

2016-08-10 Thread Gwen Shapira
I know it sounds silly, but did you check that your test setup works when you don't change the clock? This pattern can happen when two consumers somehow block each other (for example, one thread with two consumers) - so one waits for the other to join, but the other is blocked, so the first is tim