Re: [KIP-DISCUSSION] KIP-13 Quotas

2015-03-11 Thread Bhavesh Mistry
Hi Aditya, I just wanted to give you use case of rate limiting that we have implemented with producer which is a work around: Use Case 1: 1) topic based rate limiting per producer instance (not across multiple instance of producers yet, we have producer which we send Heartbeat and regular messag

[jira] [Commented] (KAFKA-1646) Improve consumer read performance for Windows

2015-03-11 Thread Jay Kreps (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1646?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14358125#comment-14358125 ] Jay Kreps commented on KAFKA-1646: -- Hey [~waldenchen] this patch is adding a TON of windo

[jira] [Updated] (KAFKA-1646) Improve consumer read performance for Windows

2015-03-11 Thread Jay Kreps (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1646?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jay Kreps updated KAFKA-1646: - Reviewer: (was: Jay Kreps) > Improve consumer read performance for Windows > ---

Re: [KIP-DISCUSSION] KIP-13 Quotas

2015-03-11 Thread Jay Kreps
Hey Todd, Yeah it is kind of weird to do the quota check after taking a request, but since the penalty is applied during that request and it just delays you to the right rate, I think it isn't exactly wrong. I admit it is weird, though. What you say about closing the connection makes sense. The i

Re: [KIP-DISCUSSION] KIP-13 Quotas

2015-03-11 Thread Jay Kreps
1. Cool 2. Yeah I just wanted to flag the dependency/interaction. 3. Cool, I think we are in agreement then that a pluggable system could possibly be nice but we can get to know it operationally before deciding to expose such a thing. 4. Yeah, I agree, let's do it as a separate discussion. We ac

[jira] [Comment Edited] (KAFKA-1646) Improve consumer read performance for Windows

2015-03-11 Thread Honghai Chen (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1646?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14358096#comment-14358096 ] Honghai Chen edited comment on KAFKA-1646 at 3/12/15 4:48 AM: --

[jira] [Commented] (KAFKA-1646) Improve consumer read performance for Windows

2015-03-11 Thread Honghai Chen (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1646?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14358096#comment-14358096 ] Honghai Chen commented on KAFKA-1646: - Het, [~jkreps] Would you like help check the re

Re: 0.8.3 release plan

2015-03-11 Thread Jay Kreps
With regard to mm, I was kind of assuming just based on the amount of work that that would go in for sure, but yeah I agree it is important. -Jay On Wed, Mar 11, 2015 at 9:39 PM, Jay Kreps wrote: > What I was trying to say was let's do a real release whenever either > consumer or authn is done

Re: 0.8.3 release plan

2015-03-11 Thread Jay Kreps
What I was trying to say was let's do a real release whenever either consumer or authn is done whichever happens first (or both if they can happen close together)--not sure which is more likely to slip. WRT the beta thing I think the question for people is whether the beta period was helpful or no

Re: 0.8.3 release plan

2015-03-11 Thread Gwen Shapira
So basically you are suggesting - lets do a beta release whenever we feel the new consumer is done? This can definitely work. I'd prefer holding for MM improvements too. IMO, its not just more improvements like flush() and compression optimization. Current MirrorMaker can lose data, which makes i

[jira] [Updated] (KAFKA-1930) Move server over to new metrics library

2015-03-11 Thread Aditya Auradkar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1930?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Aditya Auradkar updated KAFKA-1930: --- Assignee: Aditya Auradkar > Move server over to new metrics library >

[jira] [Commented] (KAFKA-1930) Move server over to new metrics library

2015-03-11 Thread Aditya Auradkar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1930?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14358077#comment-14358077 ] Aditya Auradkar commented on KAFKA-1930: I plan to work on this ticket since this

Re: 0.8.3 release plan

2015-03-11 Thread Jay Kreps
Yeah the real question is always what will we block on? I don't think we should try to hold back smaller changes. In this bucket I would include most things you described: mm improvements, replica assignment tool improvements, flush, purgatory improvements, compression optimization, etc. Likely th

[jira] [Commented] (KAFKA-1546) Automate replica lag tuning

2015-03-11 Thread Aditya Auradkar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1546?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14358074#comment-14358074 ] Aditya Auradkar commented on KAFKA-1546: I'll write a short KIP on this and circul

[jira] [Updated] (KAFKA-1831) Producer does not provide any information about which host the data was sent to

2015-03-11 Thread Joe Stein (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1831?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Joe Stein updated KAFKA-1831: - Fix Version/s: 0.8.2.0 > Producer does not provide any information about which host the data was sent > t

[jira] [Updated] (KAFKA-1852) OffsetCommitRequest can commit offset on unknown topic

2015-03-11 Thread Joe Stein (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1852?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Joe Stein updated KAFKA-1852: - Fix Version/s: 0.8.3 > OffsetCommitRequest can commit offset on unknown topic > --

[jira] [Updated] (KAFKA-1865) Add a flush() call to the new producer API

2015-03-11 Thread Joe Stein (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1865?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Joe Stein updated KAFKA-1865: - Fix Version/s: 0.8.3 > Add a flush() call to the new producer API > --

[jira] [Updated] (KAFKA-1914) Count TotalProduceRequestRate and TotalFetchRequestRate in BrokerTopicMetrics

2015-03-11 Thread Joe Stein (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1914?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Joe Stein updated KAFKA-1914: - Fix Version/s: 0.8.3 > Count TotalProduceRequestRate and TotalFetchRequestRate in BrokerTopicMetrics > ---

[jira] [Updated] (KAFKA-1910) Refactor KafkaConsumer

2015-03-11 Thread Joe Stein (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1910?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Joe Stein updated KAFKA-1910: - Fix Version/s: 0.8.3 > Refactor KafkaConsumer > -- > > Key: KAFKA-1910

[jira] [Updated] (KAFKA-1959) Class CommitThread overwrite group of Thread class causing compile errors

2015-03-11 Thread Joe Stein (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1959?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Joe Stein updated KAFKA-1959: - Fix Version/s: 0.8.3 > Class CommitThread overwrite group of Thread class causing compile errors > ---

[jira] [Updated] (KAFKA-1925) Coordinator Node Id set to INT_MAX breaks coordinator metadata updates

2015-03-11 Thread Joe Stein (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1925?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Joe Stein updated KAFKA-1925: - Fix Version/s: 0.8.3 > Coordinator Node Id set to INT_MAX breaks coordinator metadata updates > --

[jira] [Updated] (KAFKA-1943) Producer request failure rate should not include MessageSetSizeTooLarge and MessageSizeTooLargeException

2015-03-11 Thread Joe Stein (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1943?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Joe Stein updated KAFKA-1943: - Fix Version/s: 0.8.3 > Producer request failure rate should not include MessageSetSizeTooLarge and > Mess

[jira] [Updated] (KAFKA-1957) code doc typo

2015-03-11 Thread Joe Stein (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1957?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Joe Stein updated KAFKA-1957: - Fix Version/s: 0.8.3 > code doc typo > - > > Key: KAFKA-1957 >

[jira] [Updated] (KAFKA-1938) [doc] Quick start example should reference appropriate Kafka version

2015-03-11 Thread Joe Stein (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1938?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Joe Stein updated KAFKA-1938: - Fix Version/s: 0.8.3 > [doc] Quick start example should reference appropriate Kafka version >

[jira] [Updated] (KAFKA-1948) kafka.api.consumerTests are hanging

2015-03-11 Thread Joe Stein (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1948?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Joe Stein updated KAFKA-1948: - Fix Version/s: 0.8.3 > kafka.api.consumerTests are hanging > --- > >

[jira] [Updated] (KAFKA-1975) testGroupConsumption occasionally hang

2015-03-11 Thread Joe Stein (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1975?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Joe Stein updated KAFKA-1975: - Fix Version/s: 0.8.3 > testGroupConsumption occasionally hang > -- > >

[jira] [Updated] (KAFKA-1964) testPartitionReassignmentCallback hangs occasionally

2015-03-11 Thread Joe Stein (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1964?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Joe Stein updated KAFKA-1964: - Fix Version/s: 0.8.3 > testPartitionReassignmentCallback hangs occasionally > ---

[jira] [Updated] (KAFKA-1960) .gitignore does not exclude test generated files and folders.

2015-03-11 Thread Joe Stein (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1960?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Joe Stein updated KAFKA-1960: - Fix Version/s: 0.8.3 > .gitignore does not exclude test generated files and folders. > ---

[jira] [Updated] (KAFKA-1969) NPE in unit test for new consumer

2015-03-11 Thread Joe Stein (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1969?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Joe Stein updated KAFKA-1969: - Fix Version/s: 0.8.3 > NPE in unit test for new consumer > - > >

[jira] [Updated] (KAFKA-1986) Producer request failure rate should not include InvalidMessageSizeException and OffsetOutOfRangeException

2015-03-11 Thread Joe Stein (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1986?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Joe Stein updated KAFKA-1986: - Fix Version/s: 0.8.3 > Producer request failure rate should not include InvalidMessageSizeException > and

[jira] [Updated] (KAFKA-2001) OffsetCommitTest hang during setup

2015-03-11 Thread Joe Stein (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2001?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Joe Stein updated KAFKA-2001: - Fix Version/s: 0.8.3 > OffsetCommitTest hang during setup > -- > >

[jira] [Updated] (KAFKA-2009) Fix UncheckedOffset.removeOffset synchronization and trace logging issue in mirror maker

2015-03-11 Thread Joe Stein (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2009?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Joe Stein updated KAFKA-2009: - Fix Version/s: 0.8.3 > Fix UncheckedOffset.removeOffset synchronization and trace logging issue in > mirr

[jira] [Commented] (KAFKA-1546) Automate replica lag tuning

2015-03-11 Thread Jay Kreps (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1546?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14358063#comment-14358063 ] Jay Kreps commented on KAFKA-1546: -- Well iiuc the wonderfulness of this feature is that i

[jira] [Updated] (KAFKA-1546) Automate replica lag tuning

2015-03-11 Thread Joe Stein (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1546?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Joe Stein updated KAFKA-1546: - Fix Version/s: 0.8.3 > Automate replica lag tuning > --- > > Key:

Re: 0.8.3 release plan

2015-03-11 Thread Gwen Shapira
If we are going in terms of features, I can see the following features getting in in the next month or two: * New consumer * Improved Mirror Maker (I've seen tons of interest) * Centralized admin requests (aka KIP-4) * Nicer replica-reassignment tool * SSL (and perhaps also SASL)? I think this co

[jira] [Commented] (KAFKA-1546) Automate replica lag tuning

2015-03-11 Thread Joe Stein (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1546?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14358058#comment-14358058 ] Joe Stein commented on KAFKA-1546: -- we could also mark the JIRA as a bug instead of impro

Re: [VOTE] KIP-15 add a close method with timeout to KafkaProducer

2015-03-11 Thread Jay Kreps
Yeah guys, I'd like to second that. I'd really really love to get the quality of these to the point where we could broadly solicit user input and use them as a permanent document of the alternatives and rationale. I know it is a little painful to have process, but I think we all saw what happened

[jira] [Commented] (KAFKA-1461) Replica fetcher thread does not implement any back-off behavior

2015-03-11 Thread Joe Stein (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1461?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14358056#comment-14358056 ] Joe Stein commented on KAFKA-1461: -- I personally think it is over kill but i bring it up

[jira] [Commented] (KAFKA-1546) Automate replica lag tuning

2015-03-11 Thread Joe Stein (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1546?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14358054#comment-14358054 ] Joe Stein commented on KAFKA-1546: -- if folks are going to read the KIP to understand for

Re: [VOTE] KIP-15 add a close method with timeout to KafkaProducer

2015-03-11 Thread Joe Stein
Could the KIP confluence please have updated the discussion thread link, thanks... could you also remove the template boilerplate at the top "*This page is meant as a template ..*" so we can capture it for the release cleanly. Also I don't really/fully understand how this is different than flush(t

Re: [VOTE] KIP-15 add a close method with timeout to KafkaProducer

2015-03-11 Thread Harsha
+1 non-binding -Harsha On Wed, Mar 11, 2015, at 08:24 PM, Guozhang Wang wrote: > +1 (binding) > > On Wed, Mar 11, 2015 at 8:10 PM, Jiangjie Qin > wrote: > > > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-15+-+Add+a+close+method+with+a+timeout+in+the+producer > > > > > > > -- > -

Re: 0.8.3 release plan

2015-03-11 Thread Jay Kreps
Yeah I'd be in favor of a quicker, smaller release but I think as long as we have these big things in flight we should probably keep the release criteria feature-based rather than time-based, though (e.g. "when X works" not "every other month). Ideally the next release would have at least a "beta"

Re: [VOTE] KIP-15 add a close method with timeout to KafkaProducer

2015-03-11 Thread Guozhang Wang
+1 (binding) On Wed, Mar 11, 2015 at 8:10 PM, Jiangjie Qin wrote: > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-15+-+Add+a+close+method+with+a+timeout+in+the+producer > > -- -- Guozhang

Re: Review Request 31967: Patch for KAFKA-1546

2015-03-11 Thread Aditya Auradkar
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/31967/ --- (Updated March 12, 2015, 3:17 a.m.) Review request for kafka. Bugs: KAFKA-154

[VOTE] KIP-15 add a close method with timeout to KafkaProducer

2015-03-11 Thread Jiangjie Qin
https://cwiki.apache.org/confluence/display/KAFKA/KIP-15+-+Add+a+close+method+with+a+timeout+in+the+producer

[jira] [Commented] (KAFKA-1461) Replica fetcher thread does not implement any back-off behavior

2015-03-11 Thread Sriharsha Chintalapani (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1461?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14358040#comment-14358040 ] Sriharsha Chintalapani commented on KAFKA-1461: --- [~charmalloc] since there a

[jira] [Commented] (KAFKA-1546) Automate replica lag tuning

2015-03-11 Thread Jay Kreps (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1546?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14358039#comment-14358039 ] Jay Kreps commented on KAFKA-1546: -- Personally I don't think it really needs a KIP, it su

0.8.3 release plan

2015-03-11 Thread Joe Stein
There hasn't been any public discussion about the 0.8.3 release plan. There seems to be a lot of work in flight, work with patches and review that could/should get committed but now just pending KIPS, work without KIP but that is in trunk already (e.g. the new Consumer) that would be the the relea

[jira] [Commented] (KAFKA-1546) Automate replica lag tuning

2015-03-11 Thread Joe Stein (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1546?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14358017#comment-14358017 ] Joe Stein commented on KAFKA-1546: -- Shouldn't we have a KIP for this? It seems like we ar

[jira] [Commented] (KAFKA-1461) Replica fetcher thread does not implement any back-off behavior

2015-03-11 Thread Joe Stein (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1461?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14358018#comment-14358018 ] Joe Stein commented on KAFKA-1461: -- Shouldn't there be a KIP for this? > Replica fetcher

[jira] [Commented] (KAFKA-1997) Refactor Mirror Maker

2015-03-11 Thread Jiangjie Qin (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1997?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14357993#comment-14357993 ] Jiangjie Qin commented on KAFKA-1997: - Updated reviewboard https://reviews.apache.org/

[jira] [Updated] (KAFKA-1997) Refactor Mirror Maker

2015-03-11 Thread Jiangjie Qin (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1997?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jiangjie Qin updated KAFKA-1997: Attachment: KAFKA-1997_2015-03-11_19:10:53.patch > Refactor Mirror Maker > - > >

Re: Review Request 31706: Patch for KAFKA-1997

2015-03-11 Thread Jiangjie Qin
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/31706/ --- (Updated March 12, 2015, 2:10 a.m.) Review request for kafka. Bugs: KAFKA-199

Re: Review Request 31706: Patch for KAFKA-1997

2015-03-11 Thread Jiangjie Qin
> On March 12, 2015, 1:22 a.m., Guozhang Wang wrote: > > Hit this unit test failure, is this relevant? > > > > -- > > > > kafka.consumer.ZookeeperConsumerConnectorTest > > > testConsumerRebalanceListener FAILED > > junit.framework.AssertionFailedError: > > expected: bu

Re: Review Request 31967: Patch for KAFKA-1546

2015-03-11 Thread Aditya Auradkar
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/31967/ --- (Updated March 12, 2015, 1:48 a.m.) Review request for kafka. Bugs: KAFKA-154

[jira] [Commented] (KAFKA-1546) Automate replica lag tuning

2015-03-11 Thread Aditya A Auradkar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1546?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14357956#comment-14357956 ] Aditya A Auradkar commented on KAFKA-1546: -- Updated reviewboard https://reviews.a

[jira] [Updated] (KAFKA-1546) Automate replica lag tuning

2015-03-11 Thread Aditya A Auradkar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1546?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Aditya A Auradkar updated KAFKA-1546: - Attachment: KAFKA-1546_2015-03-11_18:48:09.patch > Automate replica lag tuning > -

Re: Review Request 31967: Patch for KAFKA-1546

2015-03-11 Thread Aditya Auradkar
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/31967/ --- (Updated March 12, 2015, 1:39 a.m.) Review request for kafka. Bugs: KAFKA-154

[jira] [Updated] (KAFKA-1546) Automate replica lag tuning

2015-03-11 Thread Aditya A Auradkar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1546?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Aditya A Auradkar updated KAFKA-1546: - Attachment: KAFKA-1546.patch > Automate replica lag tuning > --- >

[jira] [Commented] (KAFKA-1546) Automate replica lag tuning

2015-03-11 Thread Aditya A Auradkar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1546?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14357947#comment-14357947 ] Aditya A Auradkar commented on KAFKA-1546: -- Created reviewboard https://reviews.a

Review Request 31967: Patch for KAFKA-1546

2015-03-11 Thread Aditya Auradkar
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/31967/ --- Review request for kafka. Bugs: KAFKA-1546 https://issues.apache.org/jira/b

Re: Review Request 31706: Patch for KAFKA-1997

2015-03-11 Thread Guozhang Wang
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/31706/#review76189 --- Hit this unit test failure, is this relevant? -

[jira] [Updated] (KAFKA-1461) Replica fetcher thread does not implement any back-off behavior

2015-03-11 Thread Sriharsha Chintalapani (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1461?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Sriharsha Chintalapani updated KAFKA-1461: -- Attachment: KAFKA-1461_2015-03-11_18:17:51.patch > Replica fetcher thread does n

[jira] [Commented] (KAFKA-1461) Replica fetcher thread does not implement any back-off behavior

2015-03-11 Thread Sriharsha Chintalapani (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1461?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14357927#comment-14357927 ] Sriharsha Chintalapani commented on KAFKA-1461: --- Updated reviewboard https:/

Re: Review Request 31927: Patch for KAFKA-1461

2015-03-11 Thread Sriharsha Chintalapani
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/31927/ --- (Updated March 12, 2015, 1:17 a.m.) Review request for kafka. Bugs: KAFKA-146

Re: Review Request 30809: Patch for KAFKA-1888

2015-03-11 Thread Abhishek Nigam
> On March 12, 2015, 12:13 a.m., Jiangjie Qin wrote: > > core/src/main/scala/kafka/tools/ContinuousValidationTest.java, line 183 > > > > > > This is essentially a sync approach, can we use callback to do this? > > Abh

Re: Review Request 30809: Patch for KAFKA-1888

2015-03-11 Thread Jiangjie Qin
> On March 12, 2015, 12:13 a.m., Jiangjie Qin wrote: > > core/src/main/scala/kafka/tools/ContinuousValidationTest.java, line 183 > > > > > > This is essentially a sync approach, can we use callback to do this? > > Abh

Re: Review Request 30809: Patch for KAFKA-1888

2015-03-11 Thread Abhishek Nigam
> On March 11, 2015, 11:12 p.m., Gwen Shapira wrote: > > This looks like a very good start. I think the framework is flexible enough > > to allow us to add a variety of upgrade tests. I'm looking forward to it. > > > > > > I have few comments, but mostly I'm still confused on how this will be

Jenkins build is back to normal : Kafka-trunk #423

2015-03-11 Thread Apache Jenkins Server
See

Re: Review Request 30809: Patch for KAFKA-1888

2015-03-11 Thread Abhishek Nigam
> On March 12, 2015, 12:13 a.m., Jiangjie Qin wrote: > > core/src/main/scala/kafka/tools/ContinuousValidationTest.java, line 183 > > > > > > This is essentially a sync approach, can we use callback to do this? This is

Re: Review Request 30809: Patch for KAFKA-1888

2015-03-11 Thread Jiangjie Qin
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/30809/#review76173 --- core/src/main/scala/kafka/tools/ContinuousValidationTest.java

Build failed in Jenkins: KafkaPreCommit #36

2015-03-11 Thread Apache Jenkins Server
See Changes: [wangguoz] KAFKA-1910 Follow-up again; fix ListOffsetResponse handling for the expected error codes -- [...truncated 1643 lines...] at kafka.integration.PrimitiveApiTest.setUp(

[jira] [Commented] (KAFKA-1910) Refactor KafkaConsumer

2015-03-11 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1910?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14357794#comment-14357794 ] Guozhang Wang commented on KAFKA-1910: -- Thanks Jun, incorporated the comments and com

Re: Review Request 30809: Patch for KAFKA-1888

2015-03-11 Thread Gwen Shapira
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/30809/#review76157 --- This looks like a very good start. I think the framework is flexible

Re: Review Request 31925: KAFKA-1054: Fix remaining compiler warnings

2015-03-11 Thread Jiangjie Qin
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/31925/#review76149 --- Ship it! Makes sense. Not a committer but looks good to me :) Unit

[jira] [Commented] (KAFKA-1910) Refactor KafkaConsumer

2015-03-11 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1910?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14357725#comment-14357725 ] Jun Rao commented on KAFKA-1910: Are the changes in ConsumerTest needed? The extra logging

[jira] [Updated] (KAFKA-1997) Refactor Mirror Maker

2015-03-11 Thread Jiangjie Qin (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1997?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jiangjie Qin updated KAFKA-1997: Attachment: KAFKA-1997_2015-03-11_15:20:18.patch > Refactor Mirror Maker > - > >

[jira] [Commented] (KAFKA-1997) Refactor Mirror Maker

2015-03-11 Thread Jiangjie Qin (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1997?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14357714#comment-14357714 ] Jiangjie Qin commented on KAFKA-1997: - Updated reviewboard https://reviews.apache.org/

Re: Review Request 31706: Patch for KAFKA-1997

2015-03-11 Thread Jiangjie Qin
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/31706/ --- (Updated March 11, 2015, 10:20 p.m.) Review request for kafka. Bugs: KAFKA-19

Re: Review Request 31830: Patch for KAFKA-2009

2015-03-11 Thread Onur Karaman
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/31830/#review76145 --- Ship it! Ship It! - Onur Karaman On March 11, 2015, 6:26 p.m., J

Re: Review Request 31706: Patch for KAFKA-1997

2015-03-11 Thread Jiangjie Qin
> On March 11, 2015, 8:39 p.m., Guozhang Wang wrote: > > core/src/main/scala/kafka/tools/MirrorMaker.scala, line 195 > > > > > > Should we add the index as the suffix to the consumer id in the > > consumerConfig to

Re: Review Request 31925: KAFKA-1054: Fix remaining compiler warnings

2015-03-11 Thread Blake Smith
> On March 11, 2015, 8:39 p.m., Jiangjie Qin wrote: > > Works for me but still see the following line: > > there were 12 feature warning(s); re-run with -feature for details > > I tried ./gradlew jar -feature, but it seems does not work at all. If this > > is the related issue, can we solve it i

[jira] [Commented] (KAFKA-1501) transient unit tests failures due to port already in use

2015-03-11 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1501?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14357673#comment-14357673 ] Ewen Cheslack-Postava commented on KAFKA-1501: -- [~guozhang] Yes, that's corre

[jira] [Commented] (KAFKA-1461) Replica fetcher thread does not implement any back-off behavior

2015-03-11 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1461?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14357668#comment-14357668 ] Jun Rao commented on KAFKA-1461: [~sriharsha] and [~guozhang], thinking about this a bit m

[jira] [Commented] (KAFKA-1684) Implement TLS/SSL authentication

2015-03-11 Thread Sriharsha Chintalapani (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1684?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14357660#comment-14357660 ] Sriharsha Chintalapani commented on KAFKA-1684: --- Created reviewboard https:/

[jira] [Updated] (KAFKA-1684) Implement TLS/SSL authentication

2015-03-11 Thread Sriharsha Chintalapani (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1684?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Sriharsha Chintalapani updated KAFKA-1684: -- Attachment: KAFKA-1684.patch > Implement TLS/SSL authentication > --

Review Request 31958: Patch for KAFKA-1684

2015-03-11 Thread Sriharsha Chintalapani
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/31958/ --- Review request for kafka. Bugs: KAFKA-1684 https://issues.apache.org/jira/b

Re: Review Request 31927: Patch for KAFKA-1461

2015-03-11 Thread Jun Rao
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/31927/#review76130 --- Yes, we need to set the backoff for the consumerFetcherThread as wel

[jira] [Commented] (KAFKA-2011) Rebalance with auto.leader.rebalance.enable=false

2015-03-11 Thread K Zakee (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2011?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14357643#comment-14357643 ] K Zakee commented on KAFKA-2011: Did you mean 600 secs (10 mins)? > Rebalance with auto.l

[jira] [Commented] (KAFKA-1501) transient unit tests failures due to port already in use

2015-03-11 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1501?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14357640#comment-14357640 ] Guozhang Wang commented on KAFKA-1501: -- Thanks for the patch [~ewencp]. I have starte

Re: Review Request 31927: Patch for KAFKA-1461

2015-03-11 Thread Sriharsha Chintalapani
> On March 11, 2015, 5:06 p.m., Jun Rao wrote: > > core/src/main/scala/kafka/server/KafkaConfig.scala, line 97 > > > > > > How about making the default 1000 ms? On a second thought I think we should put a longer backof

Re: Can I be added as a contributor?

2015-03-11 Thread Brock Noland
Hi, Sorry to pile on :) but could I be added as a contributor and to confluence as well? I am brocknoland on JIRA and brockn at gmail on confluence. Cheers! Brock On Wed, Mar 11, 2015 at 1:44 PM, Joe Stein wrote: > Grant, I added your perms for Confluence. > > Grayson, I couldn't find a conflue

Re: Can I be added as a contributor?

2015-03-11 Thread Joe Stein
Grant, I added your perms for Confluence. Grayson, I couldn't find a confluence account for you so couldn't give you perms. ~ Joe Stein - - - - - - - - - - - - - - - - - http://www.stealth.ly - - - - - - - - - - - - - - - - - On Tue, Mar 10, 2015 at 8:20 AM, Grant Henke wrote: > Thanks Joe.

Re: Review Request 31925: KAFKA-1054: Fix remaining compiler warnings

2015-03-11 Thread Jiangjie Qin
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/31925/#review76122 --- Works for me but still see the following line: there were 12 feature

Re: Review Request 31706: Patch for KAFKA-1997

2015-03-11 Thread Guozhang Wang
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/31706/#review76121 --- LGTM overall except one potential issue on consumer metrics collidin

[jira] [Commented] (KAFKA-1910) Refactor KafkaConsumer

2015-03-11 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1910?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14357529#comment-14357529 ] Guozhang Wang commented on KAFKA-1910: -- [~junrao] Could you take a look at the patch

[jira] [Commented] (KAFKA-2011) Rebalance with auto.leader.rebalance.enable=false

2015-03-11 Thread Jiangjie Qin (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2011?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14357509#comment-14357509 ] Jiangjie Qin commented on KAFKA-2011: - Yes, 6000 ms sounds too short. Can you try bump

[jira] [Commented] (KAFKA-2011) Rebalance with auto.leader.rebalance.enable=false

2015-03-11 Thread K Zakee (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2011?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14357426#comment-14357426 ] K Zakee commented on KAFKA-2011: 1) The zk timeout I see occurring when the controller mig

Re: [DISCUSS] KIP-6 - New reassignment partition logic for re-balancing

2015-03-11 Thread Joe Stein
Sorry for not catching up on this thread earlier, I wanted to-do this before the KIP got its updates so we could discuss if need be and not waste more time re-writing/working things that folks have issues with or such. I captured all the comments so far here with responses. << So fair assignment b

[jira] [Commented] (KAFKA-1910) Refactor KafkaConsumer

2015-03-11 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1910?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14357366#comment-14357366 ] Guozhang Wang commented on KAFKA-1910: -- Got some problems with RB, uploading the patc

  1   2   >