Re: [DISCUSS] New partitioning for better load balancing

2015-04-07 Thread Gianmarco De Francisci Morales
Hi Guozhang, Thanks for your comments. 1) Yes, ordering cannot be guaranteed in PKG. In general, algorithms that use PGK should compute commutative and associative functions of the input. If you need strict ordering (i.e., the function is not commutative) within a partition, use KG. 2) I am not

[DISCUSS] KIP-4 - Command line and centralized administrative operations (Thread 2)

2015-04-07 Thread Andrii Biletskyi
Hi all, I wasn't able to send email to our thread (it says we exceeded message size limit :)). So I'm starting the new one. Jun, Thanks again for the review. Answering your comments: 201. I'm not sure I understand how can we evolve Cluster in backward compatible way. In my understanding topic

Re: [KIP-DISCUSSION] KIP-13 Quotas

2015-04-07 Thread Jay Kreps
Totally. But is that the only use? What I wanted to flesh out was whether the goal was: 1. Expose throttling in the client metrics 2. Enable programmatic response (i.e. stop sending stuff or something like that) I think I kind of understand (1) but let's get specific on the metric we would be addi

Re: [KIP-DISCUSSION] KIP-13 Quotas

2015-04-07 Thread Tong Li
see some response inline below. Tong Li OpenStack & Kafka Community Development Building 501/B205 liton...@us.ibm.com Jay Kreps wrote on 04/07/2015 10:41:19 AM: > From: Jay Kreps > To: "dev@kafka.apache.org" > Date: 04/07/2015 10:44 AM > Subject: Re: [KIP-DISCUSSION] KIP-13 Quotas > > Totally.

Re: Few questions regarding KAFKA-1928 (reusing common network code in core)

2015-04-07 Thread Jay Kreps
Hey Gwen, Receive and Send are meant to encapsulate the idea of sending or receiving bytes. They are supposed to represent a send or receive that is in progress. The purpose of these interfaces is to allow sends and receives using transferTo when we need to do that on the server side, this require

Re: [KIP-DISCUSSION] KIP-13 Quotas

2015-04-07 Thread Jay Kreps
Hey Tong, I don't think there is any way to infer if you are being throttled unless Kafka exposes it, though. I think the question we're discussing is just whether we expose it on the server side or change the protocol to send that back to the client to expose. We are not adding any kind of genera

Re: Review Request 31366: Patch for KAFKA-1461

2015-04-07 Thread Sriharsha Chintalapani
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/31366/ --- (Updated April 7, 2015, 3:41 p.m.) Review request for kafka. Bugs: KAFKA-1461

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

2015-04-07 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-04-07_08:41:18.patch > Replica fetcher thread does n

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

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

Re: Review Request 31366: Patch for KAFKA-1461

2015-04-07 Thread Sriharsha Chintalapani
> On March 24, 2015, 10:46 p.m., Guozhang Wang wrote: > > core/src/main/scala/kafka/server/AbstractFetcherThread.scala, lines 81-86 > > > > > > Jun has a comment about the case when all partitions gets inactive, > > wh

[jira] [Commented] (KAFKA-2082) Kafka Replication ends up in a bad state

2015-04-07 Thread Evan Huus (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2082?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14483421#comment-14483421 ] Evan Huus commented on KAFKA-2082: -- At what point will it recover? I've let it run for a

RE: [KIP-DISCUSSION] KIP-13 Quotas

2015-04-07 Thread Aditya Auradkar
Our only use case currently is to expose throttling in the client metrics. I think it's better to add more information than an isThrottled flag. Initially we were thinking of exposing the amount of time the request was delayed but the percentage throttling metric Jay mentioned also sounds good.

[jira] [Commented] (KAFKA-2082) Kafka Replication ends up in a bad state

2015-04-07 Thread Sriharsha Chintalapani (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2082?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14483431#comment-14483431 ] Sriharsha Chintalapani commented on KAFKA-2082: --- [~eapache] in my runs its r

[jira] [Comment Edited] (KAFKA-2082) Kafka Replication ends up in a bad state

2015-04-07 Thread Sriharsha Chintalapani (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2082?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14483431#comment-14483431 ] Sriharsha Chintalapani edited comment on KAFKA-2082 at 4/7/15 4:22 PM: -

[jira] [Comment Edited] (KAFKA-2082) Kafka Replication ends up in a bad state

2015-04-07 Thread Sriharsha Chintalapani (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2082?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14483431#comment-14483431 ] Sriharsha Chintalapani edited comment on KAFKA-2082 at 4/7/15 4:31 PM: -

[jira] [Commented] (KAFKA-2082) Kafka Replication ends up in a bad state

2015-04-07 Thread Sriharsha Chintalapani (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2082?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14483444#comment-14483444 ] Sriharsha Chintalapani commented on KAFKA-2082: --- [~eapache] Here is the log

[jira] [Commented] (KAFKA-2082) Kafka Replication ends up in a bad state

2015-04-07 Thread Evan Huus (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2082?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14483482#comment-14483482 ] Evan Huus commented on KAFKA-2082: -- OK, yes, if I wait long enough (2-3 minutes) then it

[jira] [Comment Edited] (KAFKA-2082) Kafka Replication ends up in a bad state

2015-04-07 Thread Evan Huus (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2082?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14483482#comment-14483482 ] Evan Huus edited comment on KAFKA-2082 at 4/7/15 4:49 PM: -- OK, ye

Re: [DISCUSS] New partitioning for better load balancing

2015-04-07 Thread Guozhang Wang
I see, thanks for the clarification. Guozhang On Tue, Apr 7, 2015 at 1:50 AM, Gianmarco De Francisci Morales < g...@apache.org> wrote: > Hi Guozhang, > > Thanks for your comments. > > 1) Yes, ordering cannot be guaranteed in PKG. In general, algorithms that > use PGK should compute commutative a

Re: [KIP-DISCUSSION] KIP-13 Quotas

2015-04-07 Thread Gwen Shapira
Re (1): We have no authorization story on the metrics collected by brokers, so I assume that access to broker metrics means knowing exactly which topics exist and their throughputs. (Prath and Don, correct me if I got it wrong...) Secure environments will strictly control access to this information

[jira] [Commented] (KAFKA-2059) ZookeeperConsumerConnectorTest.testBasic trasient failure

2015-04-07 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2059?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14483493#comment-14483493 ] Guozhang Wang commented on KAFKA-2059: -- Hmm, I did not try running just this test cas

Re: [KIP-DISCUSSION] KIP-13 Quotas

2015-04-07 Thread Joel Koshy
On Tue, Apr 07, 2015 at 07:41:19AM -0700, Jay Kreps wrote: > If it is just (1) I think the question is whether it really helps much to > have the metric on the client vs the server. I suppose this is a bit > environment specific. If you have a central metrics system it shouldn't > make any differen

[jira] [Commented] (KAFKA-2045) Memory Management on the consumer

2015-04-07 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2045?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14483550#comment-14483550 ] Guozhang Wang commented on KAFKA-2045: -- [~rzidane] Although we have some thing like c

[jira] [Commented] (KAFKA-2078) Getting Selector [WARN] Error in I/O with host java.io.EOFException

2015-04-07 Thread Aravind (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2078?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14483628#comment-14483628 ] Aravind commented on KAFKA-2078: No Load balancer. Will try to reproduce in localhost. I b

Re: KIP discussion Apr 7 at 11am PST

2015-04-07 Thread Tong Li
Jun Rao, Can you include the hangout link in the invitation? I could not find it anywhere. Thanks. Tong Li OpenStack & Kafka Community Development Building 501/B205 liton...@us.ibm.com From: Jun Rao To: "dev@kafka.apache.org" Date: 04/03/2015 01:27 PM Subject:KIP discu

Re: [KIP-DISCUSSION] KIP-13 Quotas

2015-04-07 Thread Sriharsha Chintalapani
Having auth  on top of metrics is going to be lot more difficult. How are we going to restrict metrics reporter which run as part of kafka server they will have access to all the metrics and they can publish to ganglia etc..  I look at the metrics as a read-only info. As you said metrics for all

Review Request 32931: Revert the changes in NoOffset error code

2015-04-07 Thread Guozhang Wang
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/32931/ --- Review request for kafka. Bugs: KAFKA-1910 https://issues.apache.org/jira/b

Re: Review Request 31366: Patch for KAFKA-1461

2015-04-07 Thread Guozhang Wang
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/31366/#review79205 --- Ship it! LGTM, Jun do you want to take another look? - Guozhang Wa

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

2015-04-07 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1910?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang updated KAFKA-1910: - Status: Patch Available (was: Reopened) > Refactor KafkaConsumer > -- > >

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

2015-04-07 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1910?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang updated KAFKA-1910: - Attachment: KAFKA-1910.patch > Refactor KafkaConsumer > -- > >

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

2015-04-07 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1910?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14483724#comment-14483724 ] Guozhang Wang commented on KAFKA-1910: -- Created reviewboard https://reviews.apache.or

Re: [KIP-DISCUSSION] KIP-13 Quotas

2015-04-07 Thread Gwen Shapira
Yeah, I was not suggesting adding auth to metrics - I think this needlessly complicates everything. But we need to assume that client developers will not have access to the broker metrics (because in secure environment they probably won't). Gwen On Tue, Apr 7, 2015 at 11:20 AM, Sriharsha Chintala

Re: [KIP-DISCUSSION] KIP-13 Quotas

2015-04-07 Thread Don Bosco Durai
Gwen I feel, we should assume the metrics server is an external system and the access to the server should be managed by the security features provided by the system. This way, it would be the Kafka System Administrator responsibility to ensure the metrics system is properly firewall¹ed or access

Re: [DISCUSS] KIP-4 - Command line and centralized administrative operations (Thread 2)

2015-04-07 Thread Andrii Biletskyi
Hi all, A summary of our discussion: 201. Q: Cluster updates in backward compatible way. A: Add topicConfigs map property and change constructor, this shouldn't break Consumer/Producer since TMR is used in NetworkClient, not directly by Consumer/Producer. 300. Q: Can we merge AlterTopic

[jira] [Commented] (KAFKA-2096) Enable keepalive socket option for broker to prevent socket leak

2015-04-07 Thread Allen Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2096?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14483849#comment-14483849 ] Allen Wang commented on KAFKA-2096: --- [~junrao], yes I would like submit a patch. One th

[jira] [Created] (KAFKA-2100) Client Error doesn't preserve or display original server error code when it is an unknown code

2015-04-07 Thread Gwen Shapira (JIRA)
Gwen Shapira created KAFKA-2100: --- Summary: Client Error doesn't preserve or display original server error code when it is an unknown code Key: KAFKA-2100 URL: https://issues.apache.org/jira/browse/KAFKA-2100

Re: Review Request 31850: Patch for KAFKA-1660

2015-04-07 Thread Jiangjie Qin
> On April 7, 2015, 1:28 a.m., Joel Koshy wrote: > > clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java, > > line 362 > > > > > > As you explained offline, the sender does not hav

Re: Is there a complete Kafka 0.8.* replication design document

2015-04-07 Thread Jun Rao
Yes, the wiki is a bit old. You can find out more about replication in the following links. http://kafka.apache.org/documentation.html#replication http://www.slideshare.net/junrao/kafka-replication-apachecon2013 #1, #2, #8. See the ZK layout in https://cwiki.apache.org/confluence/display/KAFKA/Kaf

[jira] [Created] (KAFKA-2101) Metric metadata-age is reset on a failed update

2015-04-07 Thread Tim Brooks (JIRA)
Tim Brooks created KAFKA-2101: - Summary: Metric metadata-age is reset on a failed update Key: KAFKA-2101 URL: https://issues.apache.org/jira/browse/KAFKA-2101 Project: Kafka Issue Type: Bug

[jira] [Created] (KAFKA-2102) Remove unnecessary synchronization when managing metadata

2015-04-07 Thread Tim Brooks (JIRA)
Tim Brooks created KAFKA-2102: - Summary: Remove unnecessary synchronization when managing metadata Key: KAFKA-2102 URL: https://issues.apache.org/jira/browse/KAFKA-2102 Project: Kafka Issue Type:

Review Request 32937: Patch for KAFKA-2102

2015-04-07 Thread Tim Brooks
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/32937/ --- Review request for kafka. Bugs: KAFKA-2102 https://issues.apache.org/jira/b

[jira] [Commented] (KAFKA-2102) Remove unnecessary synchronization when managing metadata

2015-04-07 Thread Tim Brooks (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2102?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14484054#comment-14484054 ] Tim Brooks commented on KAFKA-2102: --- Created reviewboard https://reviews.apache.org/r/32

[jira] [Updated] (KAFKA-2102) Remove unnecessary synchronization when managing metadata

2015-04-07 Thread Tim Brooks (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2102?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Tim Brooks updated KAFKA-2102: -- Attachment: KAFKA-2102.patch > Remove unnecessary synchronization when managing metadata > -

[jira] [Updated] (KAFKA-2102) Remove unnecessary synchronization when managing metadata

2015-04-07 Thread Tim Brooks (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2102?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Tim Brooks updated KAFKA-2102: -- Assignee: Tim Brooks Status: Patch Available (was: Open) > Remove unnecessary synchronization whe

[jira] [Updated] (KAFKA-2033) Small typo in documentation

2015-04-07 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2033?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao updated KAFKA-2033: --- Resolution: Fixed Fix Version/s: 0.8.3 Status: Resolved (was: Patch Available) Thanks for t

[jira] [Commented] (KAFKA-1993) Enable topic deletion as default

2015-04-07 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1993?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14484126#comment-14484126 ] Jun Rao commented on KAFKA-1993: Perhaps we should wait until the security feature is done

[jira] [Updated] (KAFKA-1993) Enable topic deletion as default

2015-04-07 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1993?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao updated KAFKA-1993: --- Status: Open (was: Patch Available) > Enable topic deletion as default > > >

Re: Review Request 31568: Patch for KAFKA-1989

2015-04-07 Thread Yasuhiro Matsuda
> On April 6, 2015, 10:35 p.m., Jun Rao wrote: > > core/src/main/scala/kafka/server/DelayedOperation.scala, lines 104-105 > > > > > > We probably should call forceComplete() first and only if it returns > > true, run

Re: Review Request 31568: Patch for KAFKA-1989

2015-04-07 Thread Yasuhiro Matsuda
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/31568/ --- (Updated April 7, 2015, 9:59 p.m.) Review request for kafka. Bugs: KAFKA-1989

[jira] [Updated] (KAFKA-1989) New purgatory design

2015-04-07 Thread Yasuhiro Matsuda (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1989?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Yasuhiro Matsuda updated KAFKA-1989: Attachment: KAFKA-1989_2015-04-07_14:59:33.patch > New purgatory design > --

[jira] [Commented] (KAFKA-1989) New purgatory design

2015-04-07 Thread Yasuhiro Matsuda (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1989?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14484170#comment-14484170 ] Yasuhiro Matsuda commented on KAFKA-1989: - Updated reviewboard https://reviews.apa

[jira] [Updated] (KAFKA-1992) Following KAFKA-1697, checkEnoughReplicasReachOffset doesn't need to get requiredAcks

2015-04-07 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1992?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao updated KAFKA-1992: --- Resolution: Fixed Fix Version/s: 0.8.3 Status: Resolved (was: Patch Available) Thanks for t

[jira] [Commented] (KAFKA-1992) Following KAFKA-1697, checkEnoughReplicasReachOffset doesn't need to get requiredAcks

2015-04-07 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1992?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14484196#comment-14484196 ] Gwen Shapira commented on KAFKA-1992: - oh wow, thanks :) Totally forgot about this one

Re: Review Request 31568: Patch for KAFKA-1989

2015-04-07 Thread Guozhang Wang
> On April 6, 2015, 10:35 p.m., Jun Rao wrote: > > core/src/main/scala/kafka/server/DelayedOperation.scala, lines 104-105 > > > > > > We probably should call forceComplete() first and only if it returns > > true, run

Re: Review Request 31568: Patch for KAFKA-1989

2015-04-07 Thread Guozhang Wang
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/31568/#review79263 --- - Guozhang Wang On April 7, 2015, 9:59 p.m., Yasuhiro Matsuda wrot

Re: Review Request 31568: Patch for KAFKA-1989

2015-04-07 Thread Guozhang Wang
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/31568/#review79257 --- core/src/main/scala/kafka/server/DelayedOperation.scala

Re: Review Request 32931: Revert the changes in NoOffset error code

2015-04-07 Thread Joel Koshy
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/32931/#review79262 --- Ship it! clients/src/main/java/org/apache/kafka/clients/consumer/i

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

2015-04-07 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1461?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang updated KAFKA-1461: - Resolution: Fixed Status: Resolved (was: Patch Available) > Replica fetcher thread does n

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

2015-04-07 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1461?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14484221#comment-14484221 ] Guozhang Wang commented on KAFKA-1461: -- Thanks for the patch, committed to trunk. >

Re: Review Request 31369: Patch for KAFKA-1982

2015-04-07 Thread Jun Rao
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/31369/#review77523 --- Sorry for the late review. A few more comments below. clients/src/

Re: Review Request 31568: Patch for KAFKA-1989

2015-04-07 Thread Yasuhiro Matsuda
> On April 7, 2015, 10:17 p.m., Guozhang Wang wrote: > > core/src/main/scala/kafka/server/DelayedOperation.scala, lines 178-188 > > > > > > What is the usage of watchCreated here? It seems only gets intialized > > as

[jira] [Commented] (KAFKA-1342) Slow controlled shutdowns can result in stale shutdown requests

2015-04-07 Thread Todd Palino (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1342?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14484278#comment-14484278 ] Todd Palino commented on KAFKA-1342: Bump I think we need to revive this. We have a "

Review Request 32942: Patch for KAFKA-1688

2015-04-07 Thread Parth Brahmbhatt
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/32942/ --- Review request for kafka. Bugs: KAFKA-1688 https://issues.apache.org/jira/b

[jira] [Commented] (KAFKA-1809) Refactor brokers to allow listening on multiple ports and IPs

2015-04-07 Thread Onur Karaman (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1809?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14484394#comment-14484394 ] Onur Karaman commented on KAFKA-1809: - In testDuplicateListeners() from KafkaConfigTes

[jira] [Commented] (KAFKA-1554) Corrupt index found on clean startup

2015-04-07 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1554?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14484427#comment-14484427 ] Gwen Shapira commented on KAFKA-1554: - Perhaps its better to rename and keep the index

[jira] [Updated] (KAFKA-345) Add a listener to ZookeeperConsumerConnector to get notified on rebalance events

2015-04-07 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-345?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira updated KAFKA-345: --- Resolution: Fixed Assignee: Jiangjie Qin Status: Resolved (was: Patch Available) Looking

[jira] [Commented] (KAFKA-1293) Mirror maker housecleaning

2015-04-07 Thread Matt Warhaftig (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1293?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14484491#comment-14484491 ] Matt Warhaftig commented on KAFKA-1293: --- Thanks everyone! Moving forward on KIP-14.

Re: Review Request 31850: Patch for KAFKA-1660

2015-04-07 Thread Jiangjie Qin
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/31850/ --- (Updated April 8, 2015, 1:18 a.m.) Review request for kafka. Bugs: KAFKA-1660

[jira] [Updated] (KAFKA-1660) Ability to call close() with a timeout on the Java Kafka Producer.

2015-04-07 Thread Jiangjie Qin (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1660?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jiangjie Qin updated KAFKA-1660: Attachment: KAFKA-1660_2015-04-07_18:18:40.patch > Ability to call close() with a timeout on the Jav

[jira] [Commented] (KAFKA-1660) Ability to call close() with a timeout on the Java Kafka Producer.

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

[jira] [Updated] (KAFKA-318) update zookeeper dependency to 3.3.5

2015-04-07 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-318?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira updated KAFKA-318: --- Resolution: Fixed Status: Resolved (was: Patch Available) We are on 3.4.6 now... > update zoo

[jira] [Updated] (KAFKA-346) Don't call commitOffsets() during rebalance

2015-04-07 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-346?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira updated KAFKA-346: --- Resolution: Fixed Status: Resolved (was: Patch Available) This is already fixed on trunk. >

[jira] [Commented] (KAFKA-1420) Replace AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK with TestUtils.createTopic in unit tests

2015-04-07 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1420?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14484516#comment-14484516 ] Gwen Shapira commented on KAFKA-1420: - This probably needs rebase. [~natty] would you

[jira] [Created] (KAFKA-2103) kafka.producer.AsyncProducerTest failure.

2015-04-07 Thread Jiangjie Qin (JIRA)
Jiangjie Qin created KAFKA-2103: --- Summary: kafka.producer.AsyncProducerTest failure. Key: KAFKA-2103 URL: https://issues.apache.org/jira/browse/KAFKA-2103 Project: Kafka Issue Type: Bug

[jira] [Updated] (KAFKA-1633) Data loss if broker is killed

2015-04-07 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1633?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira updated KAFKA-1633: Resolution: Won't Fix Status: Resolved (was: Patch Available) Looks like this issue doesn't

[jira] [Commented] (KAFKA-2078) Getting Selector [WARN] Error in I/O with host java.io.EOFException

2015-04-07 Thread Aravind (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2078?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14484527#comment-14484527 ] Aravind commented on KAFKA-2078: Jun, Not able to reproduce running on localhost. I am ru

[jira] [Updated] (KAFKA-1545) java.net.InetAddress.getLocalHost in KafkaHealthcheck.register may fail on some irregular hostnames

2015-04-07 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1545?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira updated KAFKA-1545: Status: Open (was: Patch Available) I don't see a patch... [~guozhang] - you are both assignee and

[jira] [Updated] (KAFKA-854) Upgrade dependencies for 0.8

2015-04-07 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-854?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira updated KAFKA-854: --- Resolution: Fixed Status: Resolved (was: Patch Available) already fixed > Upgrade dependencie

[jira] [Updated] (KAFKA-815) Improve SimpleConsumerShell to take in a max messages config option

2015-04-07 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-815?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira updated KAFKA-815: --- Resolution: Fixed Status: Resolved (was: Patch Available) was committed > Improve SimpleConsu

[jira] [Created] (KAFKA-2104) testDuplicateListeners() has a typo

2015-04-07 Thread Gwen Shapira (JIRA)
Gwen Shapira created KAFKA-2104: --- Summary: testDuplicateListeners() has a typo Key: KAFKA-2104 URL: https://issues.apache.org/jira/browse/KAFKA-2104 Project: Kafka Issue Type: Bug

[jira] [Commented] (KAFKA-1809) Refactor brokers to allow listening on multiple ports and IPs

2015-04-07 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1809?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14484538#comment-14484538 ] Gwen Shapira commented on KAFKA-1809: - Thanks [~onurkaraman], created KAFKA-2104 to tr

[jira] [Commented] (KAFKA-2104) testDuplicateListeners() has a typo

2015-04-07 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2104?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14484543#comment-14484543 ] Gwen Shapira commented on KAFKA-2104: - Created reviewboard https://reviews.apache.org/

[jira] [Updated] (KAFKA-2104) testDuplicateListeners() has a typo

2015-04-07 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2104?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira updated KAFKA-2104: Status: Patch Available (was: Open) > testDuplicateListeners() has a typo > -

[jira] [Updated] (KAFKA-2104) testDuplicateListeners() has a typo

2015-04-07 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2104?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira updated KAFKA-2104: Attachment: KAFKA-2104.patch > testDuplicateListeners() has a typo > -

Review Request 32953: Patch for KAFKA-2104

2015-04-07 Thread Gwen Shapira
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/32953/ --- Review request for kafka. Bugs: KAFKA-2104 https://issues.apache.org/jira/b

Re: Review Request 32953: Patch for KAFKA-2104

2015-04-07 Thread Sriharsha Chintalapani
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/32953/#review79292 --- Ship it! Ship It! - Sriharsha Chintalapani On April 8, 2015, 1:5

[jira] [Commented] (KAFKA-1993) Enable topic deletion as default

2015-04-07 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1993?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14484581#comment-14484581 ] Gwen Shapira commented on KAFKA-1993: - I think the patch is pretty neutral security-wi

[DISCUSSION] Should we enable topic deletion by default?

2015-04-07 Thread Gwen Shapira
Hi, Trying to see how the community feels about KAFKA-1993. Currently "delete topic" feature is disabled by default and must be enabled by admins. Why enable by default? * Ability to delete topics is kind of expected (and exposed in admin tools), so people are surprised when they need to tweak a

[jira] [Commented] (KAFKA-1656) "replica-assignment" option in "bin/kakfa-topics.sh" doesn't work with "alter"

2015-04-07 Thread Daisuke Kobayashi (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1656?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14484599#comment-14484599 ] Daisuke Kobayashi commented on KAFKA-1656: -- Per my understand, kafka-reassign-par

[jira] [Assigned] (KAFKA-1691) new java consumer needs ssl support as a client

2015-04-07 Thread Sriharsha Chintalapani (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1691?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Sriharsha Chintalapani reassigned KAFKA-1691: - Assignee: Sriharsha Chintalapani (was: Ivan Lyutov) > new java consumer

[jira] [Assigned] (KAFKA-1690) new java producer needs ssl support as a client

2015-04-07 Thread Sriharsha Chintalapani (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1690?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Sriharsha Chintalapani reassigned KAFKA-1690: - Assignee: Sriharsha Chintalapani (was: Ivan Lyutov) > new java producer

Re: [DISCUSSION] Should we enable topic deletion by default?

2015-04-07 Thread Jun Rao
Yes, the delete topic feature itself is much stable now. However, there are a couple of limitations. (1) If there is a running consumer, the topic will be auto-created again since currently a TopicMetadataRequest triggers a topic creation. (2) The associated consumer offsets associated with the del

[jira] [Updated] (KAFKA-1982) change kafka.examples.Producer to use the new java producer

2015-04-07 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1982?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao updated KAFKA-1982: --- Status: In Progress (was: Patch Available) Sorry for the late review. Left a few more comments on the RB. >

[jira] [Updated] (KAFKA-1954) Speed Up The Unit Tests

2015-04-07 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1954?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao updated KAFKA-1954: --- Status: In Progress (was: Patch Available) Thanks for the patch. Left a few review comments on the RB. > Spe

[jira] [Updated] (KAFKA-1973) Remove the accidentally created LogCleanerManager.scala.orig

2015-04-07 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1973?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao updated KAFKA-1973: --- Resolution: Fixed Fix Version/s: 0.8.3 Status: Resolved (was: Patch Available) Thanks for t

[jira] [Updated] (KAFKA-1953) Disambiguate metrics from different purgatories

2015-04-07 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1953?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao updated KAFKA-1953: --- Resolution: Fixed Fix Version/s: 0.8.3 Status: Resolved (was: Patch Available) Resolved thi

[jira] [Updated] (KAFKA-1907) ZkClient can block controlled shutdown indefinitely

2015-04-07 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1907?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao updated KAFKA-1907: --- Status: In Progress (was: Patch Available) > ZkClient can block controlled shutdown indefinitely > --

[jira] [Updated] (KAFKA-1906) Default the Kafka log directory to a relative path within the Kafka installation directory

2015-04-07 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1906?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao updated KAFKA-1906: --- Status: In Progress (was: Patch Available) > Default the Kafka log directory to a relative path within the Ka

[jira] [Updated] (KAFKA-1888) Add a "rolling upgrade" system test

2015-04-07 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1888?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao updated KAFKA-1888: --- Status: In Progress (was: Patch Available) > Add a "rolling upgrade" system test > --

[jira] [Updated] (KAFKA-1887) controller error message on shutting the last broker

2015-04-07 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1887?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao updated KAFKA-1887: --- Status: In Progress (was: Patch Available) > controller error message on shutting the last broker > -

  1   2   >