which zookeeper version

2014-01-01 Thread pushkar priyadarshi
Hi, I am starting a fresh deployment of kafka + zookeeper.Looking at zookeeper releases find 3.4.5 old and stable enough.Has anyone used this before in production? kafka ops wiki page says at Linkedin deployment still uses 3.3.4.Any specific reason for the same. Thanks And Regards, Pushkar

Re: only one ProducerSendThread thread when running with multiple brokers (kafka 0.8)

2014-01-01 Thread Jun Rao
When the producer send thread sends a batch of messages, it first determines which partition each message should go to. It then groups messages by broker (based on the leader of the partition of each message) and sends a produce request per broker (each request may include multiple partitions). Tho

Re: only one ProducerSendThread thread when running with multiple brokers (kafka 0.8)

2014-01-01 Thread yosi botzer
Yes I am specifying a key for each message. The same producer code works much slower when sending messages to a topic with multiple partitions comparing to a topic with a single partition. This doesn't make any sense to me at all. If I understand correctly I need multiple partitions in order to s

Re: only one ProducerSendThread thread when running with multiple brokers (kafka 0.8)

2014-01-01 Thread Jun Rao
In 0.7, we have 1 producer send thread per broker. This is changed in 0.8, where there is only 1 producer send thread per producer. If a producer needs to send messages to multiple brokers, the send thread will do that serially, which will reduce the throughput. We plan to improve that in 0.9 throu

Re: java.lang.IllegalArgumentException Buffer.limit on FetchResponse.scala + 33

2014-01-01 Thread Jun Rao
In our wire protocol, we expect the first 4 bytes for a response to be its size. If the actual size is larger than 2GB, what's stored in the those 4 bytes is the overflowed value. This could cause some of the buffer size to be smaller than it should be later on. If #partitions * fetch_size is larg

Re: only one ProducerSendThread thread when running with multiple brokers (kafka 0.8)

2014-01-01 Thread Chris Hogue
I see. If I'm following correctly you're talking about trying more application threads passing messages to the producer, which then serializes them down to a single threaded send. That sounds right, multiple producer instances in the application are the only real way to use the stock producer to g

Re: java.lang.IllegalArgumentException Buffer.limit on FetchResponse.scala + 33

2014-01-01 Thread Gerrit Jansen van Vuuren
Mm... Could be Im not sure if in a single request though. I am moving allot of data. Any pointer at were in the code the overflow might start? On 1 Jan 2014 18:13, "Jun Rao" wrote: > Are you fetching more than 2GB of data in a single fetch response (across > all partitions)? Currently, we don't h

Re: only one ProducerSendThread thread when running with multiple brokers (kafka 0.8)

2014-01-01 Thread Gerrit Jansen van Vuuren
The network is 10gbit so it seems unlikely. The 5 brokers were running without much load or probs. The bottle neck is that no matter how many threads I use for sending, the sync block in the send method will never go faster and its always limited to a single thread. I also use snappy outside and n

Re: java.lang.IllegalArgumentException Buffer.limit on FetchResponse.scala + 33

2014-01-01 Thread Jun Rao
Are you fetching more than 2GB of data in a single fetch response (across all partitions)? Currently, we don't handle integer overflow properly. Thanks, Jun On Wed, Jan 1, 2014 at 4:24 AM, Gerrit Jansen van Vuuren < gerrit...@gmail.com> wrote: > While consuming from the topics I get an IlegalA

Re: only one ProducerSendThread thread when running with multiple brokers (kafka 0.8)

2014-01-01 Thread Chris Hogue
Have you found what the actual bottleneck is? Is it the network send? Of course this would be highly influenced by the brokers' performance. After removing all compression work from the brokers we were able to get enough throughput from them that it's not really a concern. Another rough side-effec

Re: kafka + storm

2014-01-01 Thread Saulius Zemaitaitis
Hi, regarding ordering issue there are ways to avoid strict dependency on processing order as discussed in this talk: http://www.youtube.com/watch?v=ZIzqIzXhTaA by Erik Onnen from Urban Airship. On 1 January 2014 18:05, S Ahmed wrote: > I have briefly looked at storm, but just a quick question,

Re: kafka + storm

2014-01-01 Thread Joseph Lawson
If your task is completely serial you can still use Kafka and storm but you are limited in the number of workers that can apply whatever data transformation you need serialized. Basically you can use one consumer reading one Kafka partition and one storm worker being fed by the one Kafka consume

Re: only one ProducerSendThread thread when running with multiple brokers (kafka 0.8)

2014-01-01 Thread Gerrit Jansen van Vuuren
I've seen this bottle neck regardless of using compression or not, bpth situations give me poor performance on sending to kafka via the scala producer api. On 1 Jan 2014 16:42, "Chris Hogue" wrote: > Hi. > > When writing that blog we were using Kafka 0.7 as well. Understanding that > it probably

kafka + storm

2014-01-01 Thread S Ahmed
I have briefly looked at storm, but just a quick question, storm seems to have all these workers but they way it seems to me the order in which these items are processed off the queue is very random correct? In my use case order is very important so using something like storm would not be suitable

Re: only one ProducerSendThread thread when running with multiple brokers (kafka 0.8)

2014-01-01 Thread Chris Hogue
Hi. When writing that blog we were using Kafka 0.7 as well. Understanding that it probably wasn't the primary design goal, the separate send threads per broker that offered a separation of compression were a convenient side-effect of that design. We've since built new systems on 0.8 that have con

Re: only one ProducerSendThread thread when running with multiple brokers (kafka 0.8)

2014-01-01 Thread yosi botzer
This is very interesting, this is what I see as well. I wish someone could explain why it is not as explained here: http://engineering.gnip.com/kafka-async-producer/ On Wed, Jan 1, 2014 at 2:39 PM, Gerrit Jansen van Vuuren < gerrit...@gmail.com> wrote: > I don't know the code enough to comment o

Re: only one ProducerSendThread thread when running with multiple brokers (kafka 0.8)

2014-01-01 Thread Gerrit Jansen van Vuuren
I don't know the code enough to comment on that (maybe someone else on the user list can do that), but from what I've seen doing some heavy profiling I only see one thread per producer instance, it doesn't matter how many brokers or topics you have the number of threads is always 1 per producer. If

Re: only one ProducerSendThread thread when running with multiple brokers (kafka 0.8)

2014-01-01 Thread yosi botzer
But shouldn't I see a separate thread per broker (I am using the async mode)? Why do I get a better performance sending a message that has fewer partitions? On Wed, Jan 1, 2014 at 2:22 PM, Gerrit Jansen van Vuuren < gerrit...@gmail.com> wrote: > The producer is heavily synchronized (i.e. all th

java.lang.IllegalArgumentException Buffer.limit on FetchResponse.scala + 33

2014-01-01 Thread Gerrit Jansen van Vuuren
While consuming from the topics I get an IlegalArgumentException and all consumption stops, the error keeps on throwing. I've tracked it down to FectchResponse.scala line 33 The error happens when the FetchResponsePartitionData object's readFrom method calls: messageSetBuffer.limit(messageSetSize

Re: only one ProducerSendThread thread when running with multiple brokers (kafka 0.8)

2014-01-01 Thread Gerrit Jansen van Vuuren
The producer is heavily synchronized (i.e. all the code in the send method is encapsulated in one huge synchronized block). Try creating multiple producers and round robin send over them. e.g. p = producers[ n++ % producers.length ] p.send msg This will give you one thread per producer instance.

only one ProducerSendThread thread when running with multiple brokers (kafka 0.8)

2014-01-01 Thread yosi botzer
Hi, I am using kafka 0.8. I have 3 machines each running kafka broker. I am using async mode of my Producer. I expected to see 3 different threads with names starting with ProducerSendThread- (according to this article: http://engineering.gnip.com/kafka-async-producer/) However I can see only on