Re: Authorization Engine For Kafka Related to KPI-11

2015-11-03 Thread Bhavesh Mistry
+ Kafka Dev team to see if Kafka Dev team know or recommend any Auth engine for Producers/Consumers. Thanks, Bhavesh Please pardon me, I accidentally send previous blank email. On Tue, Nov 3, 2015 at 9:52 PM, Bhavesh Mistry wrote: > On Sun, Nov 1, 2015 at 11:15 PM, Bhavesh Mistry >

Re: Authorization Engine For Kafka Related to KPI-11

2015-11-03 Thread Bhavesh Mistry
On Sun, Nov 1, 2015 at 11:15 PM, Bhavesh Mistry wrote: > HI All, > > Have any one used Apache Ranger as Authorization Engine for Kafka Topic > creation, consumption (read) and write operation on a topic. I am looking > at having audit log and regulating consumption/ write to p

Re: [Discussion] KIP-34 Add Partitioner Change Listener to Partitioner Interface

2015-09-24 Thread Bhavesh Mistry
// handle partition change. > } > } > > Thanks, > > Jiangjie (Becket) Qin > > On Mon, Sep 21, 2015 at 9:13 AM, Bhavesh Mistry > wrote: > >> HI Jiagjie, >> >> Thanks for valuable feedback. >> >> 1) Thread Coordination for Change of p

Re: [Discussion] KIP-34 Add Partitioner Change Listener to Partitioner Interface

2015-09-21 Thread Bhavesh Mistry
er. > 3. If the hash of the passed in cluster is different from the hash of > cached cluster, that means a metadata refresh occurred, people can check if > there is partition change or not before do the partitioning. > > Thanks, > > Jiangjie (Becket) Qin > > On Wed, Sep

[Discussion] KIP-34 Add Partitioner Change Listener to Partitioner Interface

2015-09-16 Thread Bhavesh Mistry
Hi Kafka Dev Team, I would like you get your feedback about adding yet another method or API call to onPartitionsChange( ) to Partitioner Interface to get notify about partition changes upon metadata refresh. This will allow custom logic (implementor of Partitioner) to be notified if partition ow

Re: New Consumer API and Range Consumption with Fail-over

2015-08-05 Thread Bhavesh Mistry
bes its design: > > https://cwiki.apache.org/confluence/display/KAFKA/Kafka+0.9+Consumer+Rewrite+Design > > -Jason > > On Tue, Aug 4, 2015 at 12:01 AM, Bhavesh Mistry < > mistry.p.bhav...@gmail.com> > wrote: > > > Hi Jason and Kafka Dev Team, > > > >

Re: New Consumer API and Range Consumption with Fail-over

2015-08-04 Thread Bhavesh Mistry
t's not yet exposed in KafkaConsumer. I assume it > will be eventually, but I'm not sure whether that will be part of the > initial release. > > > Hope that helps! > > Jason > > On Thu, Jul 30, 2015 at 7:54 AM, Bhavesh Mistry < > mistry.p.bhav...@gmail.com&g

New Consumer API and Range Consumption with Fail-over

2015-07-30 Thread Bhavesh Mistry
Hello Kafka Dev Team, With new Consumer API redesign ( https://github.com/apache/kafka/blob/trunk/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java ), is there a capability to consume given the topic and partition start/ end position. How would I achieve following use

Re: [DISCUSSION] Partition Selection and Coordination By Brokers for Producers

2015-06-05 Thread Bhavesh Mistry
broker. > > Given these, I think your best bet is probably to just fix those non-java > clients to send data in a round robin way. > > Thanks, > > Jun > > On Fri, May 29, 2015 at 1:22 PM, Bhavesh Mistry < > mistry.p.bhav...@gmail.com> > wrote: > > > H

Re: [DISCUSSION] Partition Selection and Coordination By Brokers for Producers

2015-05-29 Thread Bhavesh Mistry
Hi Kafka Dev Team, I would appreciate your feedback on moving producer partition selection from producer to Broker. Also, please do let me know what is correct process of collecting feedback from Kafka Dev team and/or community. Thanks, Bhavesh On Tue, May 26, 2015 at 11:54 AM, Bhavesh

[DISCUSSION] Partition Selection and Coordination By Brokers for Producers

2015-05-26 Thread Bhavesh Mistry
Hi Kafka Dev Team, I am sorry I am new to process of discussion and/or KIP. So, I had commented other email voting chain. Please do let me know correct process for collecting and staring discussion with Kafka Dev Group. Here is original message: I have had experience with both producer and co

Re: [KIP-DISCUSSION] KIP-22 Expose a Partitioner interface in the new producer

2015-05-26 Thread Bhavesh Mistry
Hi All, This might be too late about partitioning strategy and use cases to cover. I have had experience with both producer and consumer side. I have different use case on this partition selection strategy. Problem: We have heterogeneous environment of producers (by that I mean we have n

Re: Need Access to Wiki Page To Create Page for Discussion

2015-05-07 Thread Bhavesh Mistry
, Guozhang Wang wrote: > Bhavesh, > > I could not find Bmis13 when adding you to the wiki permission. Could you > double check the account id? > > Guozhang > > On Wed, May 6, 2015 at 6:47 PM, Bhavesh Mistry > > wrote: > > > Hi Jun, > > > > Th

Re: Need Access to Wiki Page To Create Page for Discussion

2015-05-06 Thread Bhavesh Mistry
Hi Jun, The account id is Bmis13. Thanks, Bhavesh On Wed, May 6, 2015 at 4:52 PM, Jun Rao wrote: > What your wiki user id? > > Thanks, > > Jun > > On Wed, May 6, 2015 at 11:09 AM, Bhavesh Mistry < > mistry.p.bhav...@gmail.com> > wrote: > > > Hi All,

Need Access to Wiki Page To Create Page for Discussion

2015-05-06 Thread Bhavesh Mistry
Hi All, I need access to create Discussion or KIP document. Let me know what is process of getting access. Thanks, Bhavesh

Re: [DISCUSS] New consumer offset commit API

2015-04-22 Thread Bhavesh Mistry
Hi Ewen, Only time I can think of where Application needs to know result of offset was committed or not during graceful shutdown and/or Runtime.addShutdownHook() so consumer application does not get duplicated records upon restart or does not have to deal with eliminating already process offset.

Producer Behavior When one or more Brokers' Disk is Full.

2015-03-25 Thread Bhavesh Mistry
Hello Kafka Community, What is expected behavior on Producer side when one or more Brokers’ disk is full, but have not reached retention period for topics (by size or by time limit). Does producer send data to that particular brokers and/or Producer Queue gets full and always throws Queue F

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

Re: [KIP-DISCUSSION] Mirror Maker Enhancement

2015-02-25 Thread Bhavesh Mistry
ism, I know the history: we added in memory > >> >>queues > >> >> to > >> >> > > > help > >> >> > > > > > with > >> >> > > > > > > > > >>other &g

Re: [DISCUSS] KIP-8 Add a flush method to the new Java producer

2015-02-16 Thread Bhavesh Mistry
should not block and should produce the error. > > > > > > > > > > Basically the argument I am making is that the only reason you > want to > > > call > > > > > flush() is to guarantee all the sends complete so if it doesn't > > > g

Re: Kafka New(Java) Producer Connection reset by peer error and LB

2015-02-12 Thread Bhavesh Mistry
> think is what should happen (you can disable the logging in log4j if you > > don't want it). > > > > It would be nice to implement a client-side connection LRU for unused > > connections. I filed a ticket to track this: > > https://issues.apache.org/jira/browse

Re: Kafka New(Java) Producer Connection reset by peer error and LB

2015-02-10 Thread Bhavesh Mistry
roducer would continue to > function normally. Does this have any impact on the producer or is the > concern just that the exception is being logged? > > On Mon, Feb 9, 2015 at 11:21 PM, Bhavesh Mistry < > mistry.p.bhav...@gmail.com> > wrote: > > > HI Kafka Team,

Re: [DISCUSS] KIP-8 Add a flush method to the new Java producer

2015-02-09 Thread Bhavesh Mistry
> > > Bhavesh, I am wondering how you will use a flush() with a timeout since > > such a call does not actually provide any flushing guarantees? > > > > As for close(), there is a separate JIRA for this: > > > > KAFKA-1660 <https://issues.apache.org/jira/browse

Re: New consumer client

2015-02-09 Thread Bhavesh Mistry
(depending on what is corrupted). > > -jay > > On Sun, Jan 11, 2015 at 11:00 PM, Bhavesh Mistry < > mistry.p.bhav...@gmail.com > > wrote: > > > Hi Jay, > > > > One of the pain point of existing consumer code is CORRUPT_MESSAGE > > occasionally. Rig

Re: Kafka New(Java) Producer Connection reset by peer error and LB

2015-02-09 Thread Bhavesh Mistry
HI Kafka Team, Please confirm if you would like to open Jira issue to track this ? Thanks, Bhavesh On Mon, Feb 9, 2015 at 12:39 PM, Bhavesh Mistry wrote: > Hi Kakfa Team, > > We are getting this connection reset by pears after couple of minute aster > start-up of pro

Re: [DISCUSS] KIP-8 Add a flush method to the new Java producer

2015-02-09 Thread Bhavesh Mistry
Hi Jay, How about adding timeout for each method calls flush(timeout,TimeUnit) and close(timeout,TimeUNIT) ? We had runway io thread issue and caller thread should not blocked for ever for these methods ? Thanks, Bhavesh On Sun, Feb 8, 2015 at 12:18 PM, Jay Kreps wrote: > Well actually in t

Kafka New(Java) Producer Connection reset by peer error and LB

2015-02-09 Thread Bhavesh Mistry
Hi Kakfa Team, We are getting this connection reset by pears after couple of minute aster start-up of producer due to infrastructure deployment strategies we have adopted from LinkedIn. We have LB hostname and port as seed server, and all producers are getting following exception because of TCP i

Re: [KIP-DISCUSSION] Mirror Maker Enhancement

2015-02-04 Thread Bhavesh Mistry
situation where key needs to be de-serialization and get actual hashcode needs to be computed ?. Thanks, Bhavesh On Fri, Jan 30, 2015 at 1:41 PM, Jiangjie Qin wrote: > Hi Bhavesh, > > Please see inline comments. > > Jiangjie (Becket) Qin > > On 1/29/15, 7:00 PM, "Bhave

[Discussion] Producer Instance and Decouple Kafka Cluster State/TCP Connection Management

2015-02-04 Thread Bhavesh Mistry
Hi Kafka Dev team, I would like to discuss the Kafka Cluster State Management and Producer Instance relationship in 0.8.2. Current Implementation of Producer ties very closely with Kafka Cluster, and Topic Metadata management. So imagine, you have following scenarios: Application crates m

Re: [KIP-DISCUSSION] Mirror Maker Enhancement

2015-01-29 Thread Bhavesh Mistry
11:50 AM, Jiangjie Qin wrote: > Hi Bhavesh, > > I think it is the right discussion to have when we are talking about the > new new design for MM. > Please see the inline comments. > > Jiangjie (Becket) Qin > > On 1/28/15, 10:48 PM, "Bhavesh Mistry" wrote: &

Re: [KIP-DISCUSSION] Mirror Maker Enhancement

2015-01-28 Thread Bhavesh Mistry
Hi Jiangjie, I just wanted to let you know about our use case and stress the point that local data center broker cluster have fewer partitions than the destination offline broker cluster. Just because we do the batch pull from CAMUS and in order to drain data faster than the injection rate (from f

Re: [kafka-clients] Re: [VOTE] 0.8.2.0 Candidate 2 (with the correct links)

2015-01-26 Thread Bhavesh Mistry
Hi Kafka Team, I just wanted to bring this to your attention regarding Java New Producer limitation compare to old producer. a) Partition Increasing is limited to configured memory allocation. buffer.memory batch.size The maximum partition you could have before impacting (New Java Producers)

Re: Latency Tracking Across All Kafka Component

2015-01-22 Thread Bhavesh Mistry
s > > -- > > Monitoring * Alerting * Anomaly Detection * Centralized Log Management > > Solr & Elasticsearch Support * http://sematext.com/ > > > > > > On Mon, Jan 5, 2015 at 2:43 PM, Bhavesh Mistry < > mistry.p.bhav...@gmail.com > > > > > wr

Kafka Cluster Monitoring and Documentation of Internals (JMX Metrics) of Rejected Events

2015-01-12 Thread Bhavesh Mistry
Hi Kafka Team, I am trying to find out Kafka Internal and how a message can be corrupted or lost at brokers side. I have refer to following documentations for monitoring: https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Internals http://kafka.apache.org/documentation.html#monitoring I am

Re: New consumer client

2015-01-11 Thread Bhavesh Mistry
Hi Jay, One of the pain point of existing consumer code is CORRUPT_MESSAGE occasionally. Right now, it is hard to pin-point the problem of CORRUPT_MESSAGE especially when this happen on Mirror Maker side. Is there any proposal to auto skip corrupted message and have reporting visibility of CRC err

Re: Follow-up On Important Issues for 0.8.2

2015-01-08 Thread Bhavesh Mistry
Adding User Community to see if any one knows behavior of Producer for issue #1) and status of 2). Thanks, Bhavesh On Fri, Jan 2, 2015 at 12:37 PM, Bhavesh Mistry wrote: > Hi Kafka Dev Team, > > I am following-up with you guys regarding New (Java) Producer behavior in > event o

Re: Possible Memory Leak in Kafka with Tomcat

2015-01-06 Thread Bhavesh Mistry
Hi Marcel, Memory leaks will happen when there are background threads started by webapp and not shutdown (usually lib like Code hale has shutdown hook, but web app you do get to execute shutdown hook so you get memory leaks or class not found). I have faced this so you need to either use web cont

Re: Follow-up On Important Issues for 0.8.2

2015-01-05 Thread Bhavesh Mistry
c mode) when there is no issues with ZK or Brokers but network layer (firewalll or DNS issue) only. If need to be, I can file a jira ticket after understanding behavior and I can test expected behavior as well. Thanks, Bhavesh On Fri, Jan 2, 2015 at 12:37 PM, Bhavesh Mistry wrote: > Hi Kafka

Latency Tracking Across All Kafka Component

2015-01-05 Thread Bhavesh Mistry
Hi Kafka Team/Users, We are using Linked-in Kafka data pipe-line end-to-end. Producer(s) ->Local DC Brokers -> MM -> Central brokers -> Camus Job -> HDFS This is working out very well for us, but we need to have visibility of latency at each layer (Local DC Brokers -> MM -> Central brokers -> Ca

Follow-up On Important Issues for 0.8.2

2015-01-02 Thread Bhavesh Mistry
Hi Kafka Dev Team, I am following-up with you guys regarding New (Java) Producer behavior in event of network or firewall rules. I just wanted to make Java Producer resilient of any network or firewall issues, and does not become single-point of failure in application: 1) Jira Issue https://issu

[jira] [Commented] (KAFKA-1788) producer record can stay in RecordAccumulator forever if leader is no available

2014-12-30 Thread Bhavesh Mistry (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1788?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14261356#comment-14261356 ] Bhavesh Mistry commented on KAFKA-1788: --- [~parth.brahmbhatt], The close()

[jira] [Commented] (KAFKA-1788) producer record can stay in RecordAccumulator forever if leader is no available

2014-12-26 Thread Bhavesh Mistry (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1788?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14259275#comment-14259275 ] Bhavesh Mistry commented on KAFKA-1788: --- [~junrao], Please let me know

[jira] [Comment Edited] (KAFKA-1788) producer record can stay in RecordAccumulator forever if leader is no available

2014-12-26 Thread Bhavesh Mistry (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1788?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14259235#comment-14259235 ] Bhavesh Mistry edited comment on KAFKA-1788 at 12/26/14 10:3

[jira] [Commented] (KAFKA-1788) producer record can stay in RecordAccumulator forever if leader is no available

2014-12-26 Thread Bhavesh Mistry (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1788?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14259235#comment-14259235 ] Bhavesh Mistry commented on KAFKA-1788: --- The use case I have is little diffe

[jira] [Comment Edited] (KAFKA-1642) [Java New Producer Kafka Trunk] CPU Usage Spike to 100% when network connection is lost

2014-12-24 Thread Bhavesh Mistry (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1642?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14257805#comment-14257805 ] Bhavesh Mistry edited comment on KAFKA-1642 at 12/24/14 7:0

[jira] [Commented] (KAFKA-1642) [Java New Producer Kafka Trunk] CPU Usage Spike to 100% when network connection is lost

2014-12-23 Thread Bhavesh Mistry (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1642?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14257805#comment-14257805 ] Bhavesh Mistry commented on KAFKA-1642: --- [~ewencp], Thanks for patch. You

[jira] [Comment Edited] (KAFKA-1642) [Java New Producer Kafka Trunk] CPU Usage Spike to 100% when network connection is lost

2014-12-23 Thread Bhavesh Mistry (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1642?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14257686#comment-14257686 ] Bhavesh Mistry edited comment on KAFKA-1642 at 12/24/14 12:0

[jira] [Comment Edited] (KAFKA-1642) [Java New Producer Kafka Trunk] CPU Usage Spike to 100% when network connection is lost

2014-12-23 Thread Bhavesh Mistry (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1642?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14257686#comment-14257686 ] Bhavesh Mistry edited comment on KAFKA-1642 at 12/24/14 12:0

[jira] [Commented] (KAFKA-1788) producer record can stay in RecordAccumulator forever if leader is no available

2014-12-23 Thread Bhavesh Mistry (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1788?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14257691#comment-14257691 ] Bhavesh Mistry commented on KAFKA-1788: --- HI All, I did NOT try this patch,

[jira] [Comment Edited] (KAFKA-1788) producer record can stay in RecordAccumulator forever if leader is no available

2014-12-23 Thread Bhavesh Mistry (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1788?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14257691#comment-14257691 ] Bhavesh Mistry edited comment on KAFKA-1788 at 12/23/14 11:4

[jira] [Comment Edited] (KAFKA-1642) [Java New Producer Kafka Trunk] CPU Usage Spike to 100% when network connection is lost

2014-12-23 Thread Bhavesh Mistry (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1642?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14257686#comment-14257686 ] Bhavesh Mistry edited comment on KAFKA-1642 at 12/23/14 11:4

[jira] [Comment Edited] (KAFKA-1642) [Java New Producer Kafka Trunk] CPU Usage Spike to 100% when network connection is lost

2014-12-23 Thread Bhavesh Mistry (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1642?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14257686#comment-14257686 ] Bhavesh Mistry edited comment on KAFKA-1642 at 12/23/14 11:3

[jira] [Comment Edited] (KAFKA-1642) [Java New Producer Kafka Trunk] CPU Usage Spike to 100% when network connection is lost

2014-12-23 Thread Bhavesh Mistry (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1642?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14257686#comment-14257686 ] Bhavesh Mistry edited comment on KAFKA-1642 at 12/23/14 11:3

[jira] [Commented] (KAFKA-1642) [Java New Producer Kafka Trunk] CPU Usage Spike to 100% when network connection is lost

2014-12-23 Thread Bhavesh Mistry (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1642?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14257686#comment-14257686 ] Bhavesh Mistry commented on KAFKA-1642: --- [~ewencp], Patch indeed solve the

[jira] [Comment Edited] (KAFKA-1788) producer record can stay in RecordAccumulator forever if leader is no available

2014-12-16 Thread Bhavesh Mistry (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1788?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14249266#comment-14249266 ] Bhavesh Mistry edited comment on KAFKA-1788 at 12/17/14 1:2

[jira] [Commented] (KAFKA-1788) producer record can stay in RecordAccumulator forever if leader is no available

2014-12-16 Thread Bhavesh Mistry (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1788?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14249266#comment-14249266 ] Bhavesh Mistry commented on KAFKA-1788: --- [~jkreps], Can we just take quick loo

[jira] [Comment Edited] (KAFKA-1642) [Java New Producer Kafka Trunk] CPU Usage Spike to 100% when network connection is lost

2014-12-08 Thread Bhavesh Mistry (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1642?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14239063#comment-14239063 ] Bhavesh Mistry edited comment on KAFKA-1642 at 12/9/14 6:5

[jira] [Commented] (KAFKA-1642) [Java New Producer Kafka Trunk] CPU Usage Spike to 100% when network connection is lost

2014-12-08 Thread Bhavesh Mistry (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1642?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14239063#comment-14239063 ] Bhavesh Mistry commented on KAFKA-1642: --- [~stevenz3wu], 0.8.2 is very well te

[jira] [Commented] (KAFKA-1788) producer record can stay in RecordAccumulator forever if leader is no available

2014-12-04 Thread Bhavesh Mistry (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1788?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14234595#comment-14234595 ] Bhavesh Mistry commented on KAFKA-1788: --- We also need to fix the Producer C

[jira] [Commented] (KAFKA-1642) [Java New Producer Kafka Trunk] CPU Usage Spike to 100% when network connection is lost

2014-12-04 Thread Bhavesh Mistry (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1642?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14234297#comment-14234297 ] Bhavesh Mistry commented on KAFKA-1642: --- [~ewencp], 1) I will posted toward K

[jira] [Comment Edited] (KAFKA-1642) [Java New Producer Kafka Trunk] CPU Usage Spike to 100% when network connection is lost

2014-12-02 Thread Bhavesh Mistry (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1642?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14232061#comment-14232061 ] Bhavesh Mistry edited comment on KAFKA-1642 at 12/2/14 8:0

[jira] [Commented] (KAFKA-1642) [Java New Producer Kafka Trunk] CPU Usage Spike to 100% when network connection is lost

2014-12-02 Thread Bhavesh Mistry (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1642?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14232061#comment-14232061 ] Bhavesh Mistry commented on KAFKA-1642: --- Hi [~ewencp], I will not have tim

[jira] [Updated] (KAFKA-1642) [Java New Producer Kafka Trunk] CPU Usage Spike to 100% when network connection is lost

2014-11-30 Thread Bhavesh Mistry (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1642?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Bhavesh Mistry updated KAFKA-1642: -- Affects Version/s: (was: 0.8.1.1) > [Java New Producer Kafka Trunk] CPU Usage Spike to

[jira] [Commented] (KAFKA-1642) [Java New Producer Kafka Trunk] CPU Usage Spike to 100% when network connection is lost

2014-11-30 Thread Bhavesh Mistry (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1642?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14229233#comment-14229233 ] Bhavesh Mistry commented on KAFKA-1642: --- I just discovered yesterday that 0.8

[jira] [Commented] (KAFKA-1642) [Java New Producer Kafka Trunk] CPU Usage Spike to 100% when network connection is lost

2014-11-27 Thread Bhavesh Mistry (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1642?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14228040#comment-14228040 ] Bhavesh Mistry commented on KAFKA-1642: --- [~soumen.sarkar], Time out is one t

[jira] [Commented] (KAFKA-1642) [Java New Producer Kafka Trunk] CPU Usage Spike to 100% when network connection is lost

2014-11-26 Thread Bhavesh Mistry (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1642?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14226751#comment-14226751 ] Bhavesh Mistry commented on KAFKA-1642: --- [~ewencp], Even setting long follo

[jira] [Comment Edited] (KAFKA-1642) [Java New Producer Kafka Trunk] CPU Usage Spike to 100% when network connection is lost

2014-11-26 Thread Bhavesh Mistry (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1642?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14226751#comment-14226751 ] Bhavesh Mistry edited comment on KAFKA-1642 at 11/26/14 8:0

Re: [DISCUSSION] adding the serializer api back to the new java producer

2014-11-25 Thread Bhavesh Mistry
How will mix bag will work with Consumer side ? Entire site can not be rolled at once so Consumer will have to deals with New and Old Serialize Bytes ? This could be app team responsibility. Are you guys targeting 0.8.2 release, which may break customer who are already using new producer API (be

[jira] [Comment Edited] (KAFKA-1642) [Java New Producer Kafka Trunk] CPU Usage Spike to 100% when network connection is lost

2014-11-24 Thread Bhavesh Mistry (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1642?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14224041#comment-14224041 ] Bhavesh Mistry edited comment on KAFKA-1642 at 11/25/14 5:3

[jira] [Comment Edited] (KAFKA-1642) [Java New Producer Kafka Trunk] CPU Usage Spike to 100% when network connection is lost

2014-11-24 Thread Bhavesh Mistry (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1642?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14224046#comment-14224046 ] Bhavesh Mistry edited comment on KAFKA-1642 at 11/25/14 4:4

[jira] [Commented] (KAFKA-1642) [Java New Producer Kafka Trunk] CPU Usage Spike to 100% when network connection is lost

2014-11-24 Thread Bhavesh Mistry (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1642?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14224046#comment-14224046 ] Bhavesh Mistry commented on KAFKA-1642: --- Also, Are you going to port back the

[jira] [Comment Edited] (KAFKA-1642) [Java New Producer Kafka Trunk] CPU Usage Spike to 100% when network connection is lost

2014-11-24 Thread Bhavesh Mistry (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1642?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14224041#comment-14224041 ] Bhavesh Mistry edited comment on KAFKA-1642 at 11/25/14 4:4

[jira] [Comment Edited] (KAFKA-1642) [Java New Producer Kafka Trunk] CPU Usage Spike to 100% when network connection is lost

2014-11-24 Thread Bhavesh Mistry (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1642?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14224041#comment-14224041 ] Bhavesh Mistry edited comment on KAFKA-1642 at 11/25/14 4:3

[jira] [Comment Edited] (KAFKA-1642) [Java New Producer Kafka Trunk] CPU Usage Spike to 100% when network connection is lost

2014-11-24 Thread Bhavesh Mistry (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1642?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14224041#comment-14224041 ] Bhavesh Mistry edited comment on KAFKA-1642 at 11/25/14 4:3

[jira] [Commented] (KAFKA-1642) [Java New Producer Kafka Trunk] CPU Usage Spike to 100% when network connection is lost

2014-11-24 Thread Bhavesh Mistry (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1642?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14224041#comment-14224041 ] Bhavesh Mistry commented on KAFKA-1642: --- [~ewencp], I hope above steps will

[jira] [Commented] (KAFKA-1642) [Java New Producer Kafka Trunk] CPU Usage Spike to 100% when network connection is lost

2014-11-24 Thread Bhavesh Mistry (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1642?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14224039#comment-14224039 ] Bhavesh Mistry commented on KAFKA-1642: --- Here are some more cases to reproduce

[jira] [Comment Edited] (KAFKA-1642) [Java New Producer Kafka Trunk] CPU Usage Spike to 100% when network connection is lost

2014-11-24 Thread Bhavesh Mistry (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1642?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14223779#comment-14223779 ] Bhavesh Mistry edited comment on KAFKA-1642 at 11/25/14 1:3

[jira] [Commented] (KAFKA-1642) [Java New Producer Kafka Trunk] CPU Usage Spike to 100% when network connection is lost

2014-11-24 Thread Bhavesh Mistry (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1642?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14223779#comment-14223779 ] Bhavesh Mistry commented on KAFKA-1642: --- [~ewencp], Thanks for looking into

[jira] [Updated] (KAFKA-1642) [Java New Producer Kafka Trunk] CPU Usage Spike to 100% when network connection is lost

2014-11-24 Thread Bhavesh Mistry (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1642?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Bhavesh Mistry updated KAFKA-1642: -- Affects Version/s: 0.8.1.1 > [Java New Producer Kafka Trunk] CPU Usage Spike to 100% w

[jira] [Comment Edited] (KAFKA-1642) [Java New Producer Kafka Trunk] CPU Usage Spike to 100% when network connection is lost

2014-11-24 Thread Bhavesh Mistry (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1642?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14223626#comment-14223626 ] Bhavesh Mistry edited comment on KAFKA-1642 at 11/24/14 10:1

[jira] [Commented] (KAFKA-1642) [Java New Producer Kafka Trunk] CPU Usage Spike to 100% when network connection is lost

2014-11-24 Thread Bhavesh Mistry (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1642?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14223626#comment-14223626 ] Bhavesh Mistry commented on KAFKA-1642: --- Also, there is issue in my last patch

[jira] [Commented] (KAFKA-1642) [Java New Producer Kafka Trunk] CPU Usage Spike to 100% when network connection is lost

2014-11-24 Thread Bhavesh Mistry (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1642?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14223571#comment-14223571 ] Bhavesh Mistry commented on KAFKA-1642: --- Here is exact steps how to reproduce

[jira] [Comment Edited] (KAFKA-1642) [Java New Producer Kafka Trunk] CPU Usage Spike to 100% when network connection is lost

2014-11-24 Thread Bhavesh Mistry (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1642?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14223532#comment-14223532 ] Bhavesh Mistry edited comment on KAFKA-1642 at 11/24/14 9:2

[jira] [Comment Edited] (KAFKA-1642) [Java New Producer Kafka Trunk] CPU Usage Spike to 100% when network connection is lost

2014-11-24 Thread Bhavesh Mistry (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1642?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14223532#comment-14223532 ] Bhavesh Mistry edited comment on KAFKA-1642 at 11/24/14 9:2

[jira] [Commented] (KAFKA-1642) [Java New Producer Kafka Trunk] CPU Usage Spike to 100% when network connection is lost

2014-11-24 Thread Bhavesh Mistry (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1642?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14223532#comment-14223532 ] Bhavesh Mistry commented on KAFKA-1642: --- Also Regarding KafkaProder.close() me

[jira] [Comment Edited] (KAFKA-1642) [Java New Producer Kafka Trunk] CPU Usage Spike to 100% when network connection is lost

2014-11-24 Thread Bhavesh Mistry (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1642?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14223161#comment-14223161 ] Bhavesh Mistry edited comment on KAFKA-1642 at 11/24/14 6:5

[jira] [Comment Edited] (KAFKA-1642) [Java New Producer Kafka Trunk] CPU Usage Spike to 100% when network connection is lost

2014-11-24 Thread Bhavesh Mistry (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1642?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14223161#comment-14223161 ] Bhavesh Mistry edited comment on KAFKA-1642 at 11/24/14 5:2

[jira] [Commented] (KAFKA-1642) [Java New Producer Kafka Trunk] CPU Usage Spike to 100% when network connection is lost

2014-11-24 Thread Bhavesh Mistry (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1642?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14223161#comment-14223161 ] Bhavesh Mistry commented on KAFKA-1642: --- [~ewencp], The way to reproduce thi

[jira] [Comment Edited] (KAFKA-1642) [Java New Producer Kafka Trunk] CPU Usage Spike to 100% when network connection is lost

2014-11-24 Thread Bhavesh Mistry (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1642?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14223161#comment-14223161 ] Bhavesh Mistry edited comment on KAFKA-1642 at 11/24/14 5:2

[jira] [Updated] (KAFKA-1642) [Java New Producer Kafka Trunk] CPU Usage Spike to 100% when network connection is lost

2014-11-23 Thread Bhavesh Mistry (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1642?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Bhavesh Mistry updated KAFKA-1642: -- Attachment: 0001-Initial-CPU-Hish-Usage-by-Kafka-FIX-and-Also-fix-CLO.patch Please take look at

[jira] [Reopened] (KAFKA-1642) [Java New Producer Kafka Trunk] CPU Usage Spike to 100% when network connection is lost

2014-11-23 Thread Bhavesh Mistry (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1642?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Bhavesh Mistry reopened KAFKA-1642: --- The patch provided does not solve the problem. When you have more than one producer instance

[jira] [Comment Edited] (KAFKA-1642) [Java New Producer Kafka Trunk] CPU Usage Spike to 100% when network connection is lost

2014-11-23 Thread Bhavesh Mistry (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1642?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14222571#comment-14222571 ] Bhavesh Mistry edited comment on KAFKA-1642 at 11/24/14 1:3

Re: Welcome Kafka's newest committer

2014-11-20 Thread Bhavesh Mistry
Congratulation Guozhang !! Thanks for all your help and quick responses. Thanks, Bhavesh On Thu, Nov 20, 2014 at 8:23 AM, Ashish Singh wrote: > Congratulations Guozhang! > > On Thu, Nov 20, 2014 at 6:09 AM, Jarek Jarcec Cecho > wrote: > > > Congratulations Guozhang, well deserved! > > > > Ja

Re: How to recover from ConsumerRebalanceFailedException ?

2014-11-18 Thread Bhavesh Mistry
.sv.walmartlabs.com:9091/>* dies thread dies, then restart the sources. Is there any alter approach or life cycle method that so api consumer can attached to Consumer life cycle that it is dying and get notified so we can take some action. Thanks, Bhavesh On Mon, Nov 17, 2014 at 2:30 PM,

How to recover from ConsumerRebalanceFailedException ?

2014-11-17 Thread Bhavesh Mistry
Hi Kafka Team, I get following exception due to ZK/Network issues intermittently. How do I recover from consumer thread dying *programmatically* and restart source because we have alerts that due to this error we have partition OWNERSHIP is *none* ? Please let me know how to restart source and

Enforcing Network Bandwidth Quote with New Java Producer

2014-11-14 Thread Bhavesh Mistry
HI Kafka Team, We like to enforce a network bandwidth quota limit per minute on producer side. How can I do this ? I need some way to count compressed bytes on producer ? I know there is callback does not give this ability ? Let me know the best way. Thanks, Bhavesh

Re: Announcing Confluent

2014-11-06 Thread Bhavesh Mistry
HI Guys, Thanks for your awesome support. I wish you good luck !! Thanks for open sources Kafka !! Thanks, Bhavesh On Thu, Nov 6, 2014 at 10:52 AM, Rajasekar Elango wrote: > Congrats. Wish you all the very best and success. > > Thanks, > Raja. > > On Thu, Nov 6, 2014 at 1:36 PM, Niek Sande

Re: [Java New Producer] Changing Partition number and its Impact

2014-11-05 Thread Bhavesh Mistry
r the request size very > effectively which I would argue is a much bigger problem. Once you > introduce those controls you have to configure how to make use of them, > which is what this is about. > > -Jay > > On Wed, Nov 5, 2014 at 3:45 PM, Bhavesh Mistry > > wrote: >

Re: [Java New Producer] Changing Partition number and its Impact

2014-11-05 Thread Bhavesh Mistry
Hi Jay or Kafka Dev Team, Any suggestions, how I can deal with this situation of expanding partitions for New Java Producer for scalability (consumer side) ? Thanks, Bhavesh On Tue, Nov 4, 2014 at 7:08 PM, Bhavesh Mistry wrote: > Also, to added to this Old producer (Scala based in

Re: [Java New Producer] Changing Partition number and its Impact

2014-11-04 Thread Bhavesh Mistry
). Thanks, Bhavesh On Tue, Nov 4, 2014 at 4:56 PM, Bhavesh Mistry wrote: > HI Jay, > > Fundamental, problem is batch size is already configured and producers are > running in production with given configuration. ( Previous value were just > sample). How do we increase partitions

  1   2   3   >