Velocity on local machine

2014-01-10 Thread Klaus Schaefers
Hi, I am currently benchmarking Kafka against ActiveMQ and I got some results the surprised my quite a lot. ActiveMQ managed to deliver 4x more messages when running locally. But from all what I was reading I am a little bit surprised. Honestly I expected Kafka to outperform ActiveMQ. Some I am le

Re: Velocity on local machine

2014-01-10 Thread Magnus Edenhill
What performance numbers did you see? For reference you can check the following tests that were also run on the same machine as the broker: https://github.com/edenhill/librdkafka/blob/master/INTRODUCTION.md#performance-numbers Do they correspond to your numbers? Consumer thruput is not included

Re: Velocity on local machine

2014-01-10 Thread Klaus Schaefers
Hi, I have close to 2k messages per second. My machine is just a (BG 4-core i5 but I would expect more messages. I ran Kafka in the default settings. On Fri, Jan 10, 2014 at 12:31 PM, Magnus Edenhill wrote: > What performance numbers did you see? > > For reference you can check the following te

Re: Velocity on local machine

2014-01-10 Thread Gerrit Jansen van Vuuren
Have you tried using more producers. The kafka broker is performant, but the client producer's performance is not what it should be. You can also have a look at tuning the number of kafka broker's network and io threads. Regards, Gerrit On Fri, Jan 10, 2014 at 1:06 PM, Klaus Schaefers < klaus

Re: Velocity on local machine

2014-01-10 Thread Magnus Edenhill
2k msgs/s is silly, unless your messages are 10MB each, so something is clearly wrong. * What is the CPU usage and IO load when running your performance tests? * Are you using the sync producer per chance? Maybe combined with an aggressive log.flush.interval? * For reference, and to find out where

Re: custom kafka consumer - strangeness

2014-01-10 Thread Jun Rao
Are the offset used in the 2 fetch requests the same? If so, you will get the same messages twice. You consumer is responsible for advancing the offsets after consumption. Thanks, Jun On Thu, Jan 9, 2014 at 1:00 PM, Gerrit Jansen van Vuuren < gerrit...@gmail.com> wrote: > Hi, > > I'm writing a

Re: understanding OffsetOutOfRangeException's....

2014-01-10 Thread Jun Rao
Could you increase parallelism on the consumers? Thanks, Jun On Thu, Jan 9, 2014 at 1:22 PM, Jason Rosenberg wrote: > The consumption rate is a little better after the refactoring. The main > issue though, was that we had a mismatch between large and small topics. A > large topic can lag, a

Re: Velocity on local machine

2014-01-10 Thread Klaus Schaefers
Hi, during my test cpu load is quite low, roughly 50 percent, sometimes peask to 70%. >Are you using the sync producer per chance? I enforced now the async and I got a huge improvement in one of my test cases. let my explore the rest a little bit more. Cheers, Klaus On Fri, Jan 10, 2014 at

Re: Property Name Changes in Different Versions of Kafka

2014-01-10 Thread Monika Garg
Thanks a lot Joe...all my doubts/questions are cleared. Thanks again...:)

Topic not created if number of live brokers less than # replicas

2014-01-10 Thread Hanish Bansal
Hi All, As kafka is known behavior is : number of live brokers can not be less than # replicas when creating a new topic. I raised a jira (https://issues.apache.org/jira/browse/KAFKA-1182) regarding this for improvement so that topic should be created so that in case of n replication factor we c

Looks like consumer fetchers get stopped we are not getting any data

2014-01-10 Thread Seshadri, Balaji
Please let us know why we are not getting any data from Kafaka after this log from Kafka,can you guys lets us know. What could be causing all fetchers associated to be stooped why it is not doing retry. {2014-01-10 00:58:09,284} WARN [account-info-updated-hadoop-consumer_tm1mwdpl04-1389222553

RE: Looks like consumer fetchers get stopped we are not getting any data

2014-01-10 Thread Seshadri, Balaji
We also got the below error when this happens. {2014-01-10 00:58:11,292} INFO [account-info-updated-hadoop-consumer_tm1mwdpl04-1389222553159-ad59660b_watcher_executor] (?:?) - [account-info-updated-hadoop-consumer_tm1mwdpl04-1389222553159-ad59660b], exception during rebalance org.I0Itec.zkcli

RE: Looks like consumer fetchers get stopped we are not getting any data

2014-01-10 Thread Seshadri, Balaji
It would be helpful if you guys can shed some light why all fetchers are getting stopped. -Original Message- From: Seshadri, Balaji [mailto:balaji.sesha...@dish.com] Sent: Friday, January 10, 2014 11:28 AM To: users@kafka.apache.org Subject: RE: Looks like consumer fetchers get stopped w

RE: Looks like consumer fetchers get stopped we are not getting any data

2014-01-10 Thread Withers, Robert
The core problem is our consumers stop consuming and lag increases. We found this blog: https://cwiki.apache.org/confluence/display/KAFKA/FAQ#FAQ-Myconsumerseemstohavestopped,why?. This lists 3 possibilities. The blog also talks earlier about spurious rebalances, due to improper GC setting

Re: Looks like consumer fetchers get stopped we are not getting any data

2014-01-10 Thread Guozhang Wang
>From your logs the channel with the brokers are broken, are the brokers alive at that time? Guozhang On Fri, Jan 10, 2014 at 10:52 AM, Withers, Robert wrote: > The core problem is our consumers stop consuming and lag increases. We > found this blog: > https://cwiki.apache.org/confluence/displ

Re: Looks like consumer fetchers get stopped we are not getting any data

2014-01-10 Thread Guozhang Wang
Actually, the broken channel is broken by shutting down the leader-finder-thread, which is shutdown either by a rebalance retry or shutting down the consumer. Do you see "begin rebalance ..." before this log entry? And if yes, search to see if the rebalance keep failing. Guozhang On Fri, Jan 10

Re: understanding OffsetOutOfRangeException's....

2014-01-10 Thread Jason Rosenberg
well, not currently, as we don't have multiple partitions for the topics.but yes, I understand that would help too but, we are using this multiple consumers within a process approach in general with much success so far..just was curious about this ERROR I was seeing :) On Fri, Jan 10

RE: Looks like consumer fetchers get stopped we are not getting any data

2014-01-10 Thread Seshadri, Balaji
Yes rebalance begins and exceptions occurs. {2014-01-10 00:58:11,293} INFO [account-info-updated-hadoop-consumer_tm1mwdpl04-1389222553159-ad59660b_watcher_executor] (?:?) - [account-i nfo-updated-hadoop-consumer_tm1mwdpl04-1389222553159-ad59660b], Cleared the data chunks in all the consumer m

Re: Velocity on local machine

2014-01-10 Thread Neha Narkhede
What version of Kafka are you benchmarking? On Fri, Jan 10, 2014 at 8:36 AM, Klaus Schaefers < klaus.schaef...@ligatus.com> wrote: > Hi, > > during my test cpu load is quite low, roughly 50 percent, sometimes peask > to 70%. > > >Are you using the sync producer per chance? > I enforced now the a

Mirroring datacenters without vpn

2014-01-10 Thread Andrey Yegorov
Hi, I am trying to figure out best deployment plan and configuration with ops to ship new version of our system that will use kafka. Multiple geo-distributed datacenters are a given, and we are planning to build central DC to aggregate the data. Ops proposed to set up mirror to work over open int

RE: Looks like consumer fetchers get stopped we are not getting any data

2014-01-10 Thread Seshadri, Balaji
Any clue would be helpful. -Original Message- From: Seshadri, Balaji [mailto:balaji.sesha...@dish.com] Sent: Friday, January 10, 2014 12:46 PM To: users@kafka.apache.org Subject: RE: Looks like consumer fetchers get stopped we are not getting any data Yes rebalance begins and exceptions

Re: Looks like consumer fetchers get stopped we are not getting any data

2014-01-10 Thread Joel Koshy
If a consumer rebalances for any reason (e.g., if a consumer in the group has a soft failure such as a long GC) then the fetchers are stopped as part of the rebalance process. The sequence is as follows: - Stop fetchers - Commit offsets - Release partition ownership - Rebalance (i.e., figure out w

Re: Mirroring datacenters without vpn

2014-01-10 Thread Joel Koshy
> > Ops proposed to set up mirror to work over open internet channel without > secured vpn. Security of this particular data is not a concern and, as I > understood, it will give us more bandwidth (unless we buy some extra > hardware, lot's of internal details there). > > Is this configuration po

Consumers can't connect while broker is under load

2014-01-10 Thread Tim Kellogg
Hi, I have a cluster of 12 brokers receiving 10,000 msg/s from producers where each message is roughly 2.5KB. We also have 12 ZooKeepers and everything is on AWS. Under these conditions, top (the Linux utility) reports around 10-15 out of 32 for system load, so we’re at less than half capacity.

Re: Looks like consumer fetchers get stopped we are not getting any data

2014-01-10 Thread Guozhang Wang
>From the logs it seems the consumer 562b6738's registry node in Zookeeper has lost: NoNode for /consumers/account-activated-hadoop-consumer/ids/account-activated-hadoop-consumer_tm1mwdpl04-1389222557906-562b6738 As Joel suggested for now you may just stop all your consumers and restart, to debug

Re: Consumers can't connect while broker is under load

2014-01-10 Thread Guozhang Wang
Can you post the consumer logs for the long-time-to-connect scenarios? Guozhang On Fri, Jan 10, 2014 at 1:20 PM, Tim Kellogg wrote: > Hi, > > I have a cluster of 12 brokers receiving 10,000 msg/s from producers where > each message is roughly 2.5KB. We also have 12 ZooKeepers and everything is

Spring Integration Kafka support

2014-01-10 Thread Premchandra, Preetham Kukillaya
Hi, I was doing a poc using https://github.com/SpringSource/spring-integration-extensions/tree/master/spring-integration-kafka. I figured that code is expecting the brokerid=0 and ideally this will not be the case if multiple brokers are connecting to the same zookeeper. Regards Preetham This

Kafka with Docker - producer disconnecting

2014-01-10 Thread Alex Artigues
Hi everyone, I am attempting to run Zookeeper and Kafka in dockers. Both startup normally and Kafka connects ok. I think my containers are linked fine because I am able to list topics, and create topics. The producer however never delivers messages. It connects using the provided shell scripts,

Re: Topic not created if number of live brokers less than # replicas

2014-01-10 Thread Jun Rao
Hanish, Currently, we don't have plans to fix this issue in 0.8.1 since other supports such as deleting topics are probably more important. Could you work on this by pre-creating the topic when all brokers are up? Thanks, Jun On Fri, Jan 10, 2014 at 8:56 AM, Hanish Bansal < hanish.bansal.agar.

Re: understanding OffsetOutOfRangeException's....

2014-01-10 Thread Jun Rao
Do you think you can reproduce this easily? Thanks, Jun On Fri, Jan 10, 2014 at 11:33 AM, Jason Rosenberg wrote: > well, not currently, as we don't have multiple partitions for the > topics.but yes, I understand that would help too > > but, we are using this multiple consumers within

Re: Looks like consumer fetchers get stopped we are not getting any data

2014-01-10 Thread Jun Rao
Have you looked at our FAQ, especially https://cwiki.apache.org/confluence/display/KAFKA/FAQ#FAQ-Whyaretheremanyrebalancesinmyconsumerlog ? Thanks, Jun On Fri, Jan 10, 2014 at 2:25 PM, Seshadri, Balaji wrote: > Any clue would be helpful. > > -Original Message- > From: Seshadri, Balaji

Re: Looks like consumer fetchers get stopped we are not getting any data

2014-01-10 Thread Rob Withers
That was an interesting section too. Which GC settings would you suggest? Thank you, - charlie > On Jan 10, 2014, at 10:11 PM, Jun Rao wrote: > > Have you looked at our FAQ, especially > https://cwiki.apache.org/confluence/display/KAFKA/FAQ#FAQ-Whyaretheremanyrebalancesinmyconsumerlog > ? > >

Re: Kafka with Docker - producer disconnecting

2014-01-10 Thread Joe Stein
You might have to be more explicit in setting your host.name in server.properties of your brokers /*** Joe Stein Founder, Principal Consultant Big Data Open Source Security LLC http://www.stealth.ly Twitter: @allthingshadoop **

RE: Duplicate records in Kafka 0.7

2014-01-10 Thread Xuyen On
Actually, most of the duplicates I was seeing was due to a bug in an old Hive version I'm using 0.9. But I am still seeing some, although fewer duplicates. Instead of 3-13% I'm now only seeing less than 1%. This appears to be the case for each of the batch messages for my consumer which is set