Re: Log cleaner patch (KAFKA-1641) on 0.8.2.1

2015-03-16 Thread Marc Labbe
t; > Is there a way that you can reproduce this easily? > > > > Thanks, > > > > Jun > > > > On Fri, Mar 13, 2015 at 8:13 AM, Marc Labbe wrote: > > > > > No exactly, the topics are compacted but messages are not compressed. > > > &

Re: Kafka High CPU, 0.8.2.1 or openjdk?

2015-03-13 Thread Marc Labbe
llowing issue was > fixed: https://issues.apache.org/jira/browse/KAFKA-1952 > > Also see the 0.8.2.1 release notes for other fixes: > https://archive.apache.org/dist/kafka/0.8.2.1/RELEASE_NOTES.html > > > Mark > > On 13 March 2015 at 15:10, Marc Labbe wrote: > > >

Re: Log cleaner patch (KAFKA-1641) on 0.8.2.1

2015-03-13 Thread Marc Labbe
for the same reason as in the jira, i.e., > somehow compressed messages were sent to the compact topics? > > Thanks, > > Jun > > On Fri, Mar 13, 2015 at 6:45 AM, Marc Labbe wrote: > > > Hello, > > > > we're often seeing log cleaner exceptions reported

Kafka High CPU, 0.8.2.1 or openjdk?

2015-03-13 Thread Marc Labbe
Hi, our cluster is deployed on AWS, we have brokers on r3.large instances, a decent amount of topics+partitions (+600 partitions). We're not making that many requests/sec, roughly 80 produce/sec and 240 fetch/sec (not counting internal replication requests) and yet CPU hovers around 40%, which I c

Log cleaner patch (KAFKA-1641) on 0.8.2.1

2015-03-13 Thread Marc Labbe
Hello, we're often seeing log cleaner exceptions reported in KAFKA-1641 and I'd like to know if it's safe to apply the patch from that issue resolution to 0.8.2.1? Reference: https://issues.apache.org/jira/browse/KAFKA-1641 Also there are 2 patches in there, I suppose I should be using only the

Re: Adding replicas to existing topic

2014-03-25 Thread Marc Labbe
... and it works like a charm :-) On Tue, Mar 25, 2014 at 12:28 PM, Marc Labbe wrote: > Great! I'll give it a try. > > cheers, > marc > > > On Fri, Mar 21, 2014 at 1:10 PM, Neha Narkhede wrote: > >> Marc, >> >> I included the notes on increas

Re: Adding replicas to existing topic

2014-03-25 Thread Marc Labbe
t; Here you go - > > http://kafka.apache.org/081/documentation.html#basic_ops_increase_replication_factor > > Thanks, > Neha > > > On Wed, Mar 19, 2014 at 11:11 AM, Marc Labbe wrote: > > > done, https://issues.apache.org/jira/browse/KAFKA-1313 > > > > Can you expa

Re: Adding replicas to existing topic

2014-03-19 Thread Marc Labbe
hacky. Could you file a JIRA so we can get this > in 0.8.2? > > Thanks, > Neha > > > On Wed, Mar 19, 2014 at 9:10 AM, Marc Labbe wrote: > > > Hi, > > > > I have a few topics that were created a while ago with default parameters > > with only 1

Adding replicas to existing topic

2014-03-19 Thread Marc Labbe
Hi, I have a few topics that were created a while ago with default parameters with only 1 replica per partition. I am wondering if there is a way to add replicas to existing partitions of an existing topic. I didn't see anything obvious from the tools. marc

Re: Add partitions command note

2014-01-30 Thread Marc Labbe
on. That is what the warning is trying to call out. > > -Jay > > > On Thu, Jan 30, 2014 at 12:26 PM, Marc Labbe wrote: > > > Hi, > > > > I am going through the tools documentations for our own ops team. While > > doing so, I noticed the following note fr

Add partitions command note

2014-01-30 Thread Marc Labbe
Hi, I am going through the tools documentations for our own ops team. While doing so, I noticed the following note from the add partition tool: ***Please note that this tool can only be used to add partitions when data for a topic does not use a key.*** This message left me a bit dubious as to w

Re: Is there a way to delete partition at runtime?

2014-01-28 Thread Marc Labbe
cases? > The reason why we are trying to avoid shrinking partition is that it would > make the logic very complicated. For example, we need to think about > within-partition ordering guarantee with partition merging and > producing-in-progress simultaneously. > > Guozhang > &g

Re: Is there a way to delete partition at runtime?

2014-01-27 Thread Marc Labbe
I have the same need, and I've just created a Jira: https://issues.apache.org/jira/browse/KAFKA-1231 The reasoning behind it is because our topics are created on a per product basis and each of them usually starts big during the initial weeks and gradually reduces in time (1-2 years). thanks marc

Re: Apache Kafka HTTP Producer & Consumer

2014-01-16 Thread Marc Labbe
I was looking to do something like this too. A few questions maybe. I don't know what will be the use of this service so let me know if my questions are outside your scope. Isn't it a problem to create a producer instance on every request? Passing messages in the URL would be problem for me, why n

Re: Killing broker leader

2013-12-17 Thread Marc Labbe
It is worth mentioning you can reduce the likelyhood of loosing message by running the controlled shutdown before killing the broker. https://cwiki.apache.org/confluence/display/KAFKA/Replication+tools#Replicationtools-Whatiscontrolledshutdown? The connection refused is a bit surprising though. T

Re: Trade-off between topics and partitions?

2013-12-06 Thread Marc Labbe
Jun's advice is exactly what the FAQ says but maybe it should be rephrased because it is true this question comes up once in a while. Your requirement is: user must receive published messages in order. All messages published to a partition will be stored in order. A consumer reading a partition w

Re: bug in documentation?

2013-11-29 Thread Marc Labbe
while we're at it... I noticed the following typos in section 4.1 Motivation ( http://kafka.apache.org/documentation.html#majordesignelements) "we knew" instead of "we new" Finally in cases where the stream is fed into other data systems for serving we new the system would have to be able to

CSV Metrics reporter possible duplicate metrics filenames

2013-11-10 Thread Marc Labbe
HI, I sometimes use the CSV Metrics reporter to get some metrics from Kafka with more accuracy and w/o the hassle of having to configure our central metrics system to fetch the metrics from JMX, especially for one-off tests. However, every time I start a broker, I get errors from kafka like this

Re: Purgatory

2013-11-10 Thread Marc Labbe
The CSV reporter might > > run into issues though - I thought we had fixed that but could be > > wrong. > > > > Joel > > > > On Thu, Nov 07, 2013 at 11:01:06PM -0800, Joel Koshy wrote: > > > Excellent - thanks for putting that together! Will review

Re: How to use gevent with python

2013-11-10 Thread Marc Labbe
Brod totally does not work with gevent. > > Thanks > > > > > > On Mon, Nov 11, 2013 at 3:06 AM, Marc Labbe wrote: > > > Hi David, > > > > check for mahendra's fork of kafka-python, he has implemented gevent > > support in a branch ( > ht

Re: How to use gevent with python

2013-11-10 Thread Marc Labbe
Kane, you can probably achieve async consumer using the client directly (kafka.client.send_fetch_request) which a parameter to pass in a callback. I haven't used it myself. On Sun, Nov 10, 2013 at 2:33 PM, Marc Labbe wrote: > I was just browsing in the issues list and found the origin

Re: How to use gevent with python

2013-11-10 Thread Marc Labbe
I was just browsing in the issues list and found the original issue from mahendra for gevent support, https://github.com/mumrah/kafka-python/issues/37. I suggest moving the discussion there. On Sun, Nov 10, 2013 at 2:06 PM, Marc Labbe wrote: > Hi David, > > check for mahendra's

Re: How to use gevent with python

2013-11-10 Thread Marc Labbe
Hi David, check for mahendra's fork of kafka-python, he has implemented gevent support in a branch (https://github.com/mahendra/kafka-python/tree/gevent) but it hasn't made it to the main repo (yet). I am not sure why and I haven't tested it myself. For brod, AFAIK, it doesn't support 0.8 so you

RE: Purgatory

2013-11-08 Thread Marc Labbe
Re: Purgatory > > Excellent - thanks for putting that together! Will review it more > carefully tomorrow and suggest some minor edits if required. > > On Thu, Nov 07, 2013 at 10:45:40PM -0500, Marc Labbe wrote: > > I've just added a page for purgatory, feel free to comm

Re: Purgatory

2013-11-07 Thread Marc Labbe
***/ > > > On Fri, Nov 1, 2013 at 9:08 PM, Marc Labbe wrote: > > > Hi Joel, > > > > I used to have edit to the wiki, I made a few additions to it a while ago > > but it's seem I don't have it anymore. It might have been lost in the

Re: Purgatory

2013-11-01 Thread Marc Labbe
case) and we would > eventually run into an OOME. > - So we needed to periodically go through the entire watcherFor map > and explicitly remove those requests that had expired. > - More details on this are in KAFKA-664. > > Thanks, > > Joel > > [1] https://cwiki.apac

Re: Purgatory

2013-11-01 Thread Marc Labbe
Thu, Oct 31, 2013 at 2:47 PM, Guozhang Wang wrote: > > > Hello Priya, > > > > You can find the definitions of these two configs here: > > > > http://kafka.apache.org/documentation.html#brokerconfigs > > > > Guozhang > > > > > > On Thu, Oct 31,

Re: Purgatory

2013-10-31 Thread Marc Labbe
Hi Priya my understanding is producer requests will be delayed (and put in request purgatory) only if your producer uses ack=-1. It will be in the purgatory (delayed) until all brokers have acknowledged the messages to be replicated. The documentation suggests to monitor the ProducerRequestPurgato

Re: Understanding java options used by Kafka

2013-09-23 Thread Marc Labbe
> with a small heap worked better than having a large young gen and large > heap, in terms of the max GC pause. > > Hope that helps. > > Thanks, > Neha > > > On Fri, Sep 20, 2013 at 12:40 PM, Marc Labbe wrote: > > > Hi, > > > > I am

Understanding java options used by Kafka

2013-09-20 Thread Marc Labbe
Hi, I am fairly new to java and I am trying to figure out the reason behind each option listed on http://kafka.apache.org/documentation.html#java. Sorry if this a noob question :) In this list, one of option particularly struck my attention and had me wondering if the whole list of options is cor

Re: kafka.common.FailedToSendMessageException - 0.8

2013-06-20 Thread Marc Labbe
I had similar problem with my setup. It may be different for you but in my case the producer could not resolve the broker host as registered in zookeeper. You make a request to the address specified in brokers list, which successfully returns the topic metadata but the broker might have registered

Re: Arguments for Kafka over RabbitMQ ?

2013-06-07 Thread Marc Labbe
th. Queues > in Rabbit and Kafka are not the same. > > *** Rabbit does not use mnesia for message storage. It has its own > DB, optimised for messaging. You can use other DBs but this is > Complicated. > > *** Rabbit does all kinds of batching and bulk processing, and can >

Re: Arguments for Kafka over RabbitMQ ?

2013-06-06 Thread Marc Labbe
We also went through the same decision making and our arguments for Kafka where in the same lines as those Jonathan mentioned. The fact that we have heterogeneous consumers is really a deciding factor. Our requirements were to avoid loosing messages at all cost while having multiple consumers readi

Re: orders of launching kafka servers and zookeepers

2013-05-23 Thread Marc Labbe
If you merely rolling bounce a zookeeper cluster while keeping a quorum, > Kafka will recover automatically. > > Thanks, > Neha > On May 23, 2013 9:21 AM, "Marc Labbe" wrote: > > > Thanks for the answer, I was looking for this information on my side as > >

Re: orders of launching kafka servers and zookeepers

2013-05-23 Thread Marc Labbe
Thanks for the answer, I was looking for this information on my side as well. If, for some reason, the ZK cluster restarts completely, how should we deal with Kafka? Should we restart it, stop it before the ZK restart or will Kafka recover automatically? This is mainly a question for a constantly

Re: trouble loading kafka into eclipse

2013-04-12 Thread Marc Labbe
I updated the Developer setup page. Let me know if it's not clear enough or if I need to change anything. On another note, since the idea plugin is already there, would it be possible to add the sbteclipse plugin permanently as well? On Fri, Apr 12, 2013 at 10:52 AM, Jun Rao wrote: > MIS, Marc

Re: trouble loading kafka into eclipse

2013-04-12 Thread Marc Labbe
I don't know if anyone else has done that or if there is any indication against doing it but I found that adding the sbteclipse plugin in the project/plugins.sbt to be particularly easy to do and it worked for me. I am only using to look/edit the code but I am not running anything from eclipse thou

Re: Consumer re-design and Python

2013-02-15 Thread Marc Labbe
On Mon, Feb 11, 2013 at 10:31 AM, David Arthur wrote: > On 1/31/13 3:30 PM, Marc Labbe wrote: > >> Hi, >> >> I am fairly new to Kafka and Scala, I am trying to see through the >> consumer >> re-design changes, proposed and implemented for 0.8 and after, which