Re: Issue with topic deletion

2015-02-03 Thread Sumit Rangwala
On Tue, Feb 3, 2015 at 9:47 PM, Harsha wrote: > Sumit, > I grepped logs for this topic "LAX1-GRIFFIN-r13-1423001701601" > it looks like topic partitions are getting deleted in > state-change.log and this happens around 22:59 and server.log > has data till 22:5

Re: Issue with topic deletion

2015-02-03 Thread Sumit Rangwala
On Tue, Feb 3, 2015 at 9:47 PM, Harsha wrote: > Sumit, > I grepped logs for this topic "LAX1-GRIFFIN-r13-1423001701601" > it looks like topic partitions are getting deleted in > state-change.log and this happens around 22:59 and server.log > has data till 22:5

Re: Issue with topic deletion

2015-02-03 Thread Sumit Rangwala
On Tue, Feb 3, 2015 at 9:19 PM, Harsha wrote: > you are probably handling it but there is a case where you call > deleteTopic and kafka goes through delete topic process but your > consumer is running probably made a TopicMetadataRequest for the same > topic which can re-create the topic with the

Re: high cpu and network traffic when cluster has no topic

2015-02-03 Thread Guozhang Wang
Steven, you may be hitting on KAFKA-1642 . As Jay said, a bunch of such issues are fixed in the new release. Please let us know if you still see the issue with it. Guozhang On Tue, Feb 3, 2015 at 8:52 PM, Steven Wu wrote: > sure. will try my un

Re: How to fetch old messages from kafka

2015-02-03 Thread gharatmayuresh15
In that case you will have to maintain the offsets consumed and reset the offsets in case you need to consume from past. For example, suppose you have a userA for which you have a partitionA for topic TopicA. Each page shown to user increments the offset by 10. You have consumed till offset 100

Re: Issue with topic deletion

2015-02-03 Thread Harsha
Sumit, I grepped logs for this topic "LAX1-GRIFFIN-r13-1423001701601" it looks like topic partitions are getting deleted in state-change.log and this happens around 22:59 and server.log has data till 22:59. I looked for other deleted topic "LAX1-GRIFFIN-r45-1423

Re: Exception got when using csv reporter for mirror maker consumer

2015-02-03 Thread Jun Rao
Could you try removing /var/log/kafka/csv_metrics/BytesPerSec.csv first? Thanks, Jun On Tue, Feb 3, 2015 at 5:25 AM, Xiao, Luffy wrote: > Hi, > > I have checked out the trunk code and tried to use Mirror Maker. > When I enabled the csv reporter in Mirror Maker consumer config > (—consumer.conf

Re: create topic does not really executed successfully

2015-02-03 Thread Jun Rao
Could you try the 0.8.2.0 release? We did fix an issue (KAFKA-1738) that could fail the topic creation. Thanks, Jun On Mon, Feb 2, 2015 at 7:03 PM, Xinyi Su wrote: > Hi, > > -bash-4.1$ bin/kafka-topics.sh --zookeeper :2181 --create --topic > zerg.hydra --partitions 3 --replication-factor 2 >

How to fetch old messages from kafka

2015-02-03 Thread Snehalata Nagaje
Hi , We are using kafka for storing messages in chat application. Currently we divided each topic in multiple partitions. each partition stores data for given customer who uses the application. Right now on very first request, application fetches log from kafka from earliest valid offset to

Re: Issue with topic deletion

2015-02-03 Thread Harsha
you are probably handling it but there is a case where you call deleteTopic and kafka goes through delete topic process but your consumer is running probably made a TopicMetadataRequest for the same topic which can re-create the topic with the default num.partitions and replication.factor. Did you

Re: high cpu and network traffic when cluster has no topic

2015-02-03 Thread Steven Wu
sure. will try my unit test again with 0.8.2.0 release tomorrow and report back my findings. On Tue, Feb 3, 2015 at 8:42 PM, Jay Kreps wrote: > Hey Steven, > > That sounds like a bug. I think we fixed a few producer high cpu issues > since the beta, I wonder if you could repeat the same test wit

Re: high cpu and network traffic when cluster has no topic

2015-02-03 Thread Jay Kreps
Hey Steven, That sounds like a bug. I think we fixed a few producer high cpu issues since the beta, I wonder if you could repeat the same test with the 0.8.2. final release? -Jay On Tue, Feb 3, 2015 at 8:37 PM, Steven Wu wrote: > actually, my local test can reproduce the issue although not imm

Re: Issue with topic deletion

2015-02-03 Thread Sumit Rangwala
On Tue, Feb 3, 2015 at 6:48 PM, Harsha wrote: > Sumit, >lets say you are deleting a older topic "test1" do you have any >consumers running simultaneously for the topic "test1" while >deletion of topic going on. > Yes it is the case. However, after a small period of time

Re: high cpu and network traffic when cluster has no topic

2015-02-03 Thread Steven Wu
actually, my local test can reproduce the issue although not immediately. seems to happen after a few mins. I enabled TRACE level logging. here seems to be the tight loop. you can see that there are two metadata requests in one milli-seconds. kafka-producer-network-thread | foo 20:34:32,626 TRACE

high cpu and network traffic when cluster has no topic

2015-02-03 Thread Steven Wu
Hi, We have observed high cpu and high network traffic problem when 1) cluster (0.8.1.1) has no topic 2) KafkaProducer (0.8.2-beta) object is created without sending any traffic We have observed such problem twice. In both cases, problem went away immediately after one/any topic is created. Is t

Re: New Producer - ONLY sync mode?

2015-02-03 Thread Jay Kreps
Hey guys, I guess the question is whether it really matters how many underlying network requests occur? It is very hard for an application to depend on this even in the old producer since it depends on the partitions placement (a send to two partitions may go to either one machine or two and so it

Re: Issue with topic deletion

2015-02-03 Thread Harsha
Sumit, lets say you are deleting a older topic "test1" do you have any consumers running simultaneously for the topic "test1" while deletion of topic going on. -Harsha On Tue, Feb 3, 2015, at 06:17 PM, Joel Koshy wrote: > Thanks for the logs - will take a look tomorrow unless

Re: Issue with topic deletion

2015-02-03 Thread Joel Koshy
Thanks for the logs - will take a look tomorrow unless someone else gets a chance to get to it today. Joel On Tue, Feb 03, 2015 at 04:11:57PM -0800, Sumit Rangwala wrote: > On Tue, Feb 3, 2015 at 3:37 PM, Joel Koshy wrote: > > > Hey Sumit, > > > > I thought you would be providing the actual ste

[ANNOUNCEMENT] Apache Kafka 0.8.2.0 Released

2015-02-03 Thread Jun Rao
The Apache Kafka community is pleased to announce the release for Apache Kafka 0.8.2.0. The 0.8.2.0 release introduces many new features, improvements and fixes including: - A new Java producer for ease of implementation and enhanced performance. - A Kafka-based offset storage. - Delete topic s

Re: Number of Consumers Connected

2015-02-03 Thread Guozhang Wang
You can specify the consumer configs in the config file pointed by "--consumer.config Consumer config properties file.", for example: auto.commit.enable=true auto.commit.interval.ms=1000 Guozhang On Tue, Feb 3, 2015 at 4:54 PM, Sa Li wrote: > Hi, Guozhang > > Thank you very much for t

Re: Number of Consumers Connected

2015-02-03 Thread Sa Li
Hi, Guozhang Thank you very much for the reply, as you mentioned, I download the latest version https://www.apache.org/dyn/closer.cgi?path=/kafka/0.8.2-beta/kafka-0.8.2-beta-src.tgz Untar this build and here is what I see root@DO-mq-dev:/home/stuser/kafka-0.8.2-beta-src/bin# kafka-console-consu

Re: Kafka ETL Camus Question

2015-02-03 Thread Bhavesh Mistry
Hi All , Thanks for input I think I got enough information and also https://groups.google.com/forum/#!topic/camus_etl/1FcpqCnC5M4 gave me more info about the this. Thank you all for entertaining my question. I am in luck on both form :) Thanks, Bhavesh On Tue, Feb 3, 2015 at 12:56 PM, Joel

Re: Number of Consumers Connected

2015-02-03 Thread Guozhang Wang
Hello Sa, Could you try the latest 0.8.2 release, whose console consumer tool has been polished a bit with clearer properties? Guozhang On Tue, Feb 3, 2015 at 10:32 AM, Sa Li wrote: > GuoZhang > > Sorry for leaving this topic for a while, I am still not clear how to > commit the offset to zk

RE: Turning on cleanup.policy=compact for a topic => not starting cleanup ?

2015-02-03 Thread Thunder Stumpges
Nevermind, looking through the code I found the file is called "cleaner-offset-checkpoint" and exists at the root of each log.dir. Stopping each server in turn, and deleting this file, then starting back up *seems* to have fixed the error! Thanks again Joel! -Thunder -Original Message

Re: Issue with topic deletion

2015-02-03 Thread Sumit Rangwala
On Tue, Feb 3, 2015 at 3:37 PM, Joel Koshy wrote: > Hey Sumit, > > I thought you would be providing the actual steps to reproduce :) > I want to but some proprietary code prevents me to do it. > Nevertheless, can you get all the relevant logs: state change logs and > controller logs at the ver

RE: Turning on cleanup.policy=compact for a topic => not starting cleanup ?

2015-02-03 Thread Thunder Stumpges
Thanks Joel. The jira issue mentions a workaround to "remove the cleaner offset checkpoint" what exactly does that entail? Thanks again! Thunder -Original Message- From: Joel Koshy [mailto:jjkosh...@gmail.com] Sent: Tuesday, February 03, 2015 3:57 PM To: users@kafka.apache.org Subject

Re: Turning on cleanup.policy=compact for a topic => not starting cleanup ?

2015-02-03 Thread Joel Koshy
Oh yes I forgot about that possibility. I think this should be fixed by KAFKA-1641 which will be in the next release. On Tue, Feb 03, 2015 at 11:50:54PM +, Thunder Stumpges wrote: > AH hah! The log-cleaner.log file did NOT go into the logs directory for some > reason, found it in the kafka r

RE: Turning on cleanup.policy=compact for a topic => not starting cleanup ?

2015-02-03 Thread Thunder Stumpges
AH hah! The log-cleaner.log file did NOT go into the logs directory for some reason, found it in the kafka root directory. But in there I found the following output: [2015-02-03 15:43:08,128] INFO Cleaner 0: Beginning cleaning of log dev_testcompact-9. (kafka.log.LogCleaner) [2015-02-03 15:43:0

Re: Turning on cleanup.policy=compact for a topic => not starting cleanup ?

2015-02-03 Thread Joel Koshy
> > So, lastly the thread dump. Not sure exactly how to do that or what thread > I'd be looking for specifically... Found a suggestion to run > > Jstack -l > jstack.out That should work, although you can just send SIGQUIT (kill -3) to the PID. > So I did that, and looked for anything contain

Re: Issue with topic deletion

2015-02-03 Thread Joel Koshy
Hey Sumit, I thought you would be providing the actual steps to reproduce :) Nevertheless, can you get all the relevant logs: state change logs and controller logs at the very least and if possible server logs and send those over? Joel On Tue, Feb 03, 2015 at 03:27:43PM -0800, Sumit Rangwala wro

RE: Turning on cleanup.policy=compact for a topic => not starting cleanup ?

2015-02-03 Thread Thunder Stumpges
Thanks Joel for the quick turnaround! 1st, I took a look at "max-dirty-percent" and noticed it is "100", so that's a good sign that it *should* have something to clean. 2nd, I didn't realize the cleaner messages went to a different log! However I have NO log-cleaner.log file in the normal logs

Issue with topic deletion

2015-02-03 Thread Sumit Rangwala
In my setup kafka brokers are set for auto topic creation. In the scenario below a node informs other nodes (currently 5 in total) about a number of new (non-existent) topics, and all the nodes almost simultaneously open a consumer for each of those topics. Sometime later another node informs all

Re: Turning on cleanup.policy=compact for a topic => not starting cleanup ?

2015-02-03 Thread Joel Koshy
- Can you check the log cleaner logs? - Do you have any compressed messages in your log? Or messages without a key? - By default it is in a log-cleaner.log file unless you modified that. - Can you take a thread-dump to see if the log cleaner is still alive? - Also, there is an mbean that you can

Turning on cleanup.policy=compact for a topic => not starting cleanup ?

2015-02-03 Thread Thunder Stumpges
Hi guys, I am having a difficult time getting the log compaction to run on a topic I created initially with cleanup.policy=delete. Here's the details: 3 brokers, 0.8.1.1 all have log.cleaner.enable=true (other than this there are no non-defaults set in our server.properties) Create the topic fi

Re: kafka-web-console goes down regularly

2015-02-03 Thread Sa Li
Good idea, Joel, will do it now. Thanks AL On Tue, Feb 3, 2015 at 2:12 PM, Joel Koshy wrote: > Can you contact the maintainer directly? > http://github.com/claudemamo/kafka-web-console/issues > > On Tue, Feb 03, 2015 at 12:09:46PM -0800, Sa Li wrote: > > Hi, All > > > > I am currently using ka

Re: kafka-web-console goes down regularly

2015-02-03 Thread Joel Koshy
Can you contact the maintainer directly? http://github.com/claudemamo/kafka-web-console/issues On Tue, Feb 03, 2015 at 12:09:46PM -0800, Sa Li wrote: > Hi, All > > I am currently using kafka-web-console to monitor the kafka system, it get > down regularly, so I have to restart it every few hours

Re: Can't create a topic; can't delete it either

2015-02-03 Thread Joel Koshy
That would be great! Joel On Tue, Feb 03, 2015 at 01:39:02PM -0800, Sumit Rangwala wrote: > Joel, > > My apology for being silent on this thread. I have now been able to > recreate the setup in a docker environment and I can now consistently > re-create an issue with auto-topic creation. Since t

Re: Can't create a topic; can't delete it either

2015-02-03 Thread Sumit Rangwala
Joel, My apology for being silent on this thread. I have now been able to recreate the setup in a docker environment and I can now consistently re-create an issue with auto-topic creation. Since the exact setup is different I will a start another thread will the the information. Sumit On Thu, Ja

Re: Kafka ETL Camus Question

2015-02-03 Thread Joel Koshy
There was some confusion here - turns out that they do turn it on. I added Tu to this thread and his response: We have speculative set to true by default. With these settings, we are seeing about 5-7% of the tasks have speculative tasks launched, other 90% finished within the standard deviations

Re: [kafka-clients] Re: [VOTE] 0.8.2.0 Candidate 3

2015-02-03 Thread Pradeep Gollakota
Lithium Technologies would love to host you guys for a release party in SF if you guys want. :) On Tue, Feb 3, 2015 at 11:04 AM, Gwen Shapira wrote: > When's the party? > :) > > On Mon, Feb 2, 2015 at 8:13 PM, Jay Kreps wrote: > > Yay! > > > > -Jay > > > > On Mon, Feb 2, 2015 at 2:23 PM, Neha

kafka-web-console goes down regularly

2015-02-03 Thread Sa Li
Hi, All I am currently using kafka-web-console to monitor the kafka system, it get down regularly, so I have to restart it every few hours which is kinda annoying. I downloaded two versions https://github.com/claudemamo/kafka-web-console http://mungeol-heo.blogspot.ca/2014/12/kafka-web-console.ht

Re: [kafka-clients] Re: [VOTE] 0.8.2.0 Candidate 3

2015-02-03 Thread Gwen Shapira
When's the party? :) On Mon, Feb 2, 2015 at 8:13 PM, Jay Kreps wrote: > Yay! > > -Jay > > On Mon, Feb 2, 2015 at 2:23 PM, Neha Narkhede wrote: >> >> Great! Thanks Jun for helping with the release and everyone involved for >> your contributions. >> >> On Mon, Feb 2, 2015 at 1:32 PM, Joe Stein wr

Re: Number of Consumers Connected

2015-02-03 Thread Sa Li
GuoZhang Sorry for leaving this topic for a while, I am still not clear how to commit the offset to zk from commandline, I tried this bin/kafka-console-consumer.sh --zookeeper 10.100.71.33:2181 --topic pipe-test-2 --from-beginning --property pipe It seems generate a console-consumer-001 in zK, b

Re: Increased CPU usage with 0.8.2-beta

2015-02-03 Thread Jun Rao
Mathias, The new hprof doesn't reveal anything new to me. We did fix the logic in using Purgatory in 0.8.2, which could potentially drive up the CPU usage a bit. To verify that, could you do your test on a single broker (with replication factor 1) btw 0.8.1 and 0.8.2 and see if there is any signif

RE: Kafka ETL Camus Question

2015-02-03 Thread Aditya Auradkar
Hi Bhavesh, I just checked with one of the devs on the Camus team. We run the Camus job with speculative execution disabled. Aditya From: Pradeep Gollakota [pradeep...@gmail.com] Sent: Monday, February 02, 2015 11:15 PM To: users@kafka.apache.org Subject

Re: Kafka long tail latency issue

2015-02-03 Thread Jay Kreps
If you are on 0.8.1 or higher and are running with replication consider disabling the forced log flush, that will definitely lead to latency spikes as the flush is synchronous. You will still get durability from replication and the background OS flush. On Linux the background I/O flush the OS does

Re: When ZooKeeper quorum is down

2015-02-03 Thread Guozhang Wang
Hi Omid, That is an interesting question.. This paragraph was written some time ago and we have not test ZK failure / resume since, and it is hard to tell if these cases still exist or not. One thing we can do is to add different ZK quorum failure scenarios to the system test to have it covered o

Re: Kafka long tail latency issue

2015-02-03 Thread Guozhang Wang
Hi Xinyi, With ack = -1 and three replicas in ISR, the latency is bounded by the time spent on follower replica fetching from the leader most of the time, since the produce response cannot be acknowledged back until all ISR has fetched the data. You can try to reduce "replica.fetch.wait.max.ms" a

Re: Why the consumer can't receve the message if fetch_size=1

2015-02-03 Thread Yury Ruchin
The fetch request size should not be less than the maximum message size. Apparently, all your messages are larger than 1 byte, so when you set fetch size to 1, your consumer is unable to fetch anything. 2015-02-03 10:53 GMT+03:00 Honda Wei ( 魏宏達 ) : > Hi Kafka Team > > I write some simple program

Re: Logstash to Kafka

2015-02-03 Thread Yury Ruchin
This is a quote from Kafka documentation: "The routing decision is influenced by the kafka.producer.Partitioner. interface Partitioner { int partition(T key, int numPartitions); } The partition API uses the key and the number of available broker partitions to return a partition id. This id is u

Exception got when using csv reporter for mirror maker consumer

2015-02-03 Thread Xiao, Luffy
Hi, I have checked out the trunk code and tried to use Mirror Maker. When I enabled the csv reporter in Mirror Maker consumer config (—consumer.config=c1.properties) kafka.metrics.polling.interval.secs=5 kafka.metrics.reporters=kafka.metrics.KafkaCSVMetricsReporter kafka.csv.metrics.dir=/var/

Why the consumer can't receve the message if fetch_size=1

2015-02-03 Thread 魏宏達
Hi Kafka Team I write some simple program and find a strange issue. Why the consumer can’t receive the message if the fetch is set to 1 , but when the parameter set to 100. The consumer will receive the message normally. FetchRequest req = new FetchRequestBuilder() .clientId(clientName)

Re: Increased CPU usage with 0.8.2-beta

2015-02-03 Thread Mathias Söderberg
>>> > >>>>>> I upgraded a Kafka cluster from v0.8.1.1 to v0.8.2-beta and noticed > >>>> that > >>>>>> the CPU usage on the broker machines went up by roughly 40%, from > >>> ~60% > >>>> to > >>>>>> ~100% and am wondering if anyone else has experienced something > >>>> similar? > >>>>>> The load average also went up by 2x-3x. > >>>>>> > >>>>>> We're running on EC2 and the cluster currently consists of four > >>>>> m1.xlarge, > >>>>>> with roughly 1100 topics / 4000 partitions. Using Java 7 (1.7.0_65 > >>> to > >>>> be > >>>>>> exact) and Scala 2.9.2. Configurations can be found over here: > >>>>>> https://gist.github.com/mthssdrbrg/7df34a795e07eef10262. > >>>>>> > >>>>>> I'm assuming that this is not expected behaviour for 0.8.2-beta? > >>>>>> > >>>>>> Best regards, > >>>>>> Mathias > >>>>>> > >>>>> > >>>>> > >>>>> -- > >>>>> Thanks, > >>>>> Neha > >>>>> > >>> > >>> > >>> -- > >>> Thanks, > >>> Neha > >>> > > kafka-0.8.2.0-snappy-20150203.hprof.gz Description: GNU Zip compressed data

When ZooKeeper quorum is down

2015-02-03 Thread Omid Aladini
Hi, Reading the official FAQ, I bumped into this paragraph: Once the Zookeeper quorum is down, brokers could result in a bad state and > could not normally serve client requests, etc. Although when Zookeeper > quorum recovers, the Kafka brokers should be able to resume to normal state > automatic

Logstash to Kafka

2015-02-03 Thread Vineet Mishra
Hi, I am having a setup where I am sniffing some logs(ofcourse the big ones) through Logstash Forwarder and forwarding it to Logstash, which in turn publish these events to Kafka. I have created the Kafka Topic ensuring the required number of Partitions and Replication Factor but not sure with Lo

Re: Reg: Perf test

2015-02-03 Thread David Morales
Useful links to start with: http://kafka.apache.org/07/performance.html http://engineering.linkedin.com/kafka/benchmarking-apache-kafka-2-million-writes-second-three-cheap-machines 2015-02-03 10:44 GMT+01:00 Selva Kumar : > Newbie to Kafka. > > Plz let me know how/tools to measure the produ

Reg: Perf test

2015-02-03 Thread Selva Kumar
Newbie to Kafka. Plz let me know how/tools to measure the producer and consumer performance. -- Regards Selvakumar.V

Kafka long tail latency issue

2015-02-03 Thread Xinyi Su
Hi, I am building Kafka cluster and run producer perf test to get Kafka latency performance. >From test result, I notice that the long tail latency is very high and increased with time passing by although the 99.9% result looks very good. The worst latency can reach more than 1 second. Besides, dis