Issue with auto topic creation as well

2015-02-04 Thread Sumit Rangwala
I am observing the following exception with kafka client: 2015-02-04 00:17:27,345 (LAX1-GRIFFIN-r8-1423037468055-pf13797-lax1-GriffinDownloader-1423037818264_c7b1e843ff51-1423037822122-eb7afca7-leader-finder-thread) ClientUtils$ WARN: Fetching topic metadata with correlation id 112 for topics [Set

Re: How to fetch old messages from kafka

2015-02-04 Thread Snehalata Nagaje
Hi Mayuresh, Thanks for quick response. We can reset the offset and get first 10 messages, but since we need to back in reverse sequence, suppose user has consumed messages upto 100 offset , currently there are only last 10 messages are visible, from 100 -90, now I want to retrieve messages

Got IOException when writing metrics to csv file

2015-02-04 Thread tao xiao
Hi team, I was running the mirror maker off the trunk code and got IOException when configuring the mirror maker to use KafkaCSVMetricsReporter as the metric reporter Here is the exception I got java.io.IOException: Unable to create /tmp/csv1/BytesPerSec.csv at com.yammer.metrics.reporting.CsvR

kafka consumers parallel consuming at consumer level or thread level?

2015-02-04 Thread Zijing Guo
Hi,I have some question regarding how kafka consumers achieve parallel consuming for one topic. Say I have 2 partitions for topic1 and I have a consumer Group A, now:1: If no consumer under consumer Group A subscribe topic1, then no message will be delivery to this consumer group.2: If there is

Re: When ZooKeeper quorum is down

2015-02-04 Thread Omid Aladini
Sure: https://issues.apache.org/jira/browse/KAFKA-1918 Thanks! Omid On Tue, Feb 3, 2015 at 5:32 PM, Guozhang Wang wrote: > 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

How to configure "asynchronous replication"

2015-02-04 Thread 陈洪海
Hi Kafka, The doc mentioned that “Single producer thread, 3x ASYNChronous replication”, how to configure it? What’s the configuration item? Or need use async produce api to send? https://engineering.linkedin.com/kafka/benchmarking-apache-kafka-2-million-writes-second-three-che

Re: How to configure "asynchronous replication"

2015-02-04 Thread Joe Stein
Hi Chen, Take a look at the quick start https://kafka.apache.org/documentation.html#quickstart step 3 change --replication-factor 1 to be --replication-factor 3 would do that. You need to have at least 3 brokers running live when you do that. You also want to look at changing your partition count

Re: New Producer - ONLY sync mode?

2015-02-04 Thread Joe Stein
Now that 0.8.2.0 is in the wild I look forward to working with more and seeing what folks start to-do with this function https://dist.apache.org/repos/dist/release/kafka/0.8.2.0/javadoc/org/apache/kafka/clients/producer/KafkaProducer.html#send(org.apache.kafka.clients.producer.ProducerRecord, org.a

Re: kafka consumers parallel consuming at consumer level or thread level?

2015-02-04 Thread Guozhang Wang
Hi Edwin, 1. Yes. 2. Yes. 3. Yes; and there is no difference in terms of parallelism. In the new consumer client (org.apache.kafka.clients.consumer.KafkaConsumer) that is going to be out in the next release, each consumer instance is single-threaded, i.e. it will only have one thread for fetching

Topic migration mentionning one removed server

2015-02-04 Thread Anthony Pastor
Hello, I'm trying to understand an awkward issue we're having on our Kafka production. We currently have 8 Kafka servers named: kafka1, kafka2, kafka3 ... Few weeks ago we migrated all topics present on kafka7 and then have shutdown it. The main issue is a performance issue on Kafka5. Topic migr

Re: New Producer - ONLY sync mode?

2015-02-04 Thread Jay Kreps
Yeah totally. Using a callback is, of course, the Right Thing for this kind of stuff. But I have found that kind of asynchronous thinking can be hard for people. Even if you get out of the pre-java 8 syntactic pain that anonymous inner classes inflict just dealing with multiple threads of control w

Re: kafka consumers parallel consuming at consumer level or thread level?

2015-02-04 Thread Zijing Guo
Hi Guozhang,Thanks for your clarification for this, it's start to making sense now.Edwin On Wednesday, February 4, 2015 11:04 AM, Guozhang Wang wrote: Hi Edwin, 1. Yes. 2. Yes. 3. Yes; and there is no difference in terms of parallelism. In the new consumer client (org.apache.kafka

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

2015-02-04 Thread Steven Wu
I have re-run my unit test with 0.8.2.0. same tight-loop problem happened after a few mins. On Tue, Feb 3, 2015 at 10:00 PM, Guozhang Wang wrote: > Steven, you may be hitting on KAFKA-1642 > . > > As Jay said, a bunch of such issues are fixed in

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

2015-02-04 Thread Jay Kreps
That's a bug then, filed https://issues.apache.org/jira/browse/KAFKA-1919 -Jay On Wed, Feb 4, 2015 at 9:01 AM, Steven Wu wrote: > I have re-run my unit test with 0.8.2.0. same tight-loop problem happened > after a few mins. > > On Tue, Feb 3, 2015 at 10:00 PM, Guozhang Wang wrote: > > > Steven

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

2015-02-04 Thread Bhavesh Mistry
Hi Steven, Can you please try to see if io thread is indeed a problem ? The following on works on Linux: ps -p "$java_pid" -L -o tid,pcpu jstack -F "$java_pid" Then compare the thread # (may have to Hex # to decimal) between the Jstack and ps command. This will tell you which thread is consu

Re: [ANNOUNCEMENT] Apache Kafka 0.8.2.0 Released

2015-02-04 Thread Rajasekar Elango
YaY!. Thanks to Jun and everybody who contributed to this release. We have been waiting for this release for a while. Thanks, Rajasekar Elango (Salesforce.com). On Tue, Feb 3, 2015 at 8:37 PM, Jun Rao wrote: > The Apache Kafka community is pleased to announce the release for Apache > Kafka 0.8.

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

2015-02-04 Thread Steven Wu
Bhavesh, this is on Mac OS. I couldn't get similar options to make ps/jstack work on Mac. will continue to try if I can make them work. logging output does show "kafka-producer-network-thread" sends two metadata requests per milli-seconds. Thanks, Steven On Wed, Feb 4, 2015 at 9:15 AM, Bhavesh

error handling with high-level consumer

2015-02-04 Thread Steven Wu
Hi, We have observed these two exceptions with consumer *iterator.next()* recently. want to ask how should we handle them properly. *1) CRC corruption* Message is corrupt (stored crc = 433657556, computed crc = 3265543163) I assume in this case we should just catch it and move on to the next msg

Re: New Producer - ONLY sync mode?

2015-02-04 Thread Steve Morin
Looking at this thread I would ideally want something at least the right recipe to mimic sync behavior like Otis is talking about. In the second case, would like to be able to individually know if messages have failed even regardless if they are in separate batches, sort of like what Kinesis does

Re: New Producer - ONLY sync mode?

2015-02-04 Thread Steven Wu
this is the jira regarding blocking on metadata. https://issues.apache.org/jira/browse/KAFKA-1835 I am less concerned about the first-time blocking. I am more concerned about the situation when kafka cluster/brokers are completely down. now we can screw up the producer apps. I hope that we can tak

fetchrequest and assigned replica issue

2015-02-04 Thread Karts
Hey guys Seeing the following set of errors when sending messages to the client via the API. Thing is 4 out of 5 messages get in, and only one message is being lost, and I haven’t been able to pinpoint the culprit. Listed below are the only set of errors I see on my kafka out logs. Any advice is g

Re: Issue with topic deletion

2015-02-04 Thread Sumit Rangwala
> > >> I have since stopped the container so I cannot say if > LAX1-GRIFFIN-r45-142388317 was one of the topic in "marked for > deletion" forever. However, there were many topics (at least 10 of them) > that were perennially in "marked for deletion" state. > > I have the setup to recreate the

Re: New Producer - ONLY sync mode?

2015-02-04 Thread Gwen Shapira
I thought Jay Kreps had the right recipes: To mimic the old Sync producer: producer.send(record).get(); To mimic old batches: List responses = new ArrayList(); for(input: recordBatch) responses.add(producer.send(input)); for(response: responses) response.get Perhaps we need to add this

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

2015-02-04 Thread Steven Wu
Bhavesh, unfortunately, ps cmd in Mac doesn't display thread id. I tried DTrace, but it only shows kernel thread id (not Java thread id). anyway, I updated the jira with producer metrics. it clearly shows request rate shoot up to 18K/sec. Thanks, Steven On Wed, Feb 4, 2015 at 9:48 AM, Steven Wu

Re: metric-kafka problems

2015-02-04 Thread Sa Li
Hello, Joe Continue this thread, I got following monitoring tools on my DEV, 1. graphite + statsD 2. kafka-web-console 3. JMX + jconsole 4. kafkaOffsetMonitor 5. Kafka Manager (yahoo just open source it) They all work fine locally on my dev, but I am thinking to get it work out of box cause I jus

Re: Detecting lost connection in high level consumer

2015-02-04 Thread harikiran
Hey! Did anyone get a chance to read this? Thanks!! On Wed, Jan 28, 2015 at 6:21 PM harikiran wrote: > > Hi > > I am using the 0811 Kafka High level consumer and I have configured " > consumer.timeout.ms" to a value that is not -1, say 5000ms. > > I create the consumer iterator and invoke hasNex

kafka out-of-box monitoring system

2015-02-04 Thread Sa Li
Hello, all I've recently played around the different kafka monitoring tools, got following monitoring tools on my DEV, 1. graphite + statsD 2. kafka-web-console 3. JMX + jconsole 4. kafkaOffsetMonitor 5. Kafka Manager (yahoo just open source it) They all work fine locally on my dev, but I am thi

Re: Detecting lost connection in high level consumer

2015-02-04 Thread harikiran
Thanks Guozhang and Jun for your replies. On Wed, Feb 4, 2015 at 4:07 PM, harikiran wrote: > Hey! Did anyone get a chance to read this? > > Thanks!! > On Wed, Jan 28, 2015 at 6:21 PM harikiran wrote: > >> >> Hi >> >> I am using the 0811 Kafka High level consumer and I have configured " >> consu

Re: Issue with topic deletion

2015-02-04 Thread Joel Koshy
I took a look at your logs. I agree with Harsh that the logs seem truncated. The basic issue though is that you have session expirations and controller failover. Broker 49554 was the controller and hosted some partition(s) of LAX1-GRIFFIN-r13-1423001701601. After controller failover the new control

Kafka producer perf script throw java.io.IOException

2015-02-04 Thread Xinyi Su
Hi, Today I updated Kafka cluster from 0.8.2-beta to 0.8.2.0 and run kafka producer performance test. The test cannot continue because of some exceptions thrown which does not occur at 0.8.2-beta. My perf library is kafka-perf_2.9.2-0.8.0.jar which is the latest version on maven repository. -bas

Re: create topic does not really executed successfully

2015-02-04 Thread Xinyi Su
Yeah, I will upgrade to 0.8.2.0 and try again. On 4 February 2015 at 13:41, Jun Rao wrote: > 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-

Re: Kafka producer perf script throw java.io.IOException

2015-02-04 Thread tao xiao
Hi, In order to get it work you can turn off csv-reporter. On Thu, Feb 5, 2015 at 1:06 PM, Xinyi Su wrote: > Hi, > > Today I updated Kafka cluster from 0.8.2-beta to 0.8.2.0 and run kafka > producer performance test. > > The test cannot continue because of some exceptions thrown which does not

Re: Issue with auto topic creation as well

2015-02-04 Thread Jun Rao
Could you try the 0.8.2.0 release? It fixed one issue related to topic creation. Thanks, Jun On Wed, Feb 4, 2015 at 12:58 AM, Sumit Rangwala wrote: > I am observing the following exception with kafka client: > > 2015-02-04 00:17:27,345 > > (LAX1-GRIFFIN-r8-1423037468055-pf13797-lax1-GriffinDow

Re: Topic migration mentionning one removed server

2015-02-04 Thread Jun Rao
Which version of Kafka are you using? Anything interesting from the controller log? Thanks, Jun On Wed, Feb 4, 2015 at 8:19 AM, Anthony Pastor wrote: > Hello, > > I'm trying to understand an awkward issue we're having on our Kafka > production. > We currently have 8 Kafka servers named: > kafk

Re: error handling with high-level consumer

2015-02-04 Thread Jun Rao
1) Does the corruption happen to console consumer as well? If so, could you run DumpLogSegment tool to see if the data is corrupted on disk? Thanks, Jun On Wed, Feb 4, 2015 at 9:55 AM, Steven Wu wrote: > Hi, > > We have observed these two exceptions with consumer *iterator.next()* > recently.

Re: fetchrequest and assigned replica issue

2015-02-04 Thread Jun Rao
Which version of Kafka are you using? Thanks, Jun On Wed, Feb 4, 2015 at 11:45 AM, Karts wrote: > Hey guys > > Seeing the following set of errors when sending messages to the client via > the API. Thing is 4 out of 5 messages get in, and only one message is being > lost, and I haven’t been abl

Re: Issue with topic deletion

2015-02-04 Thread Sumit Rangwala
On Wed, Feb 4, 2015 at 6:14 PM, Joel Koshy wrote: > I took a look at your logs. I agree with Harsh that the logs seem > truncated. The basic issue though is that you have session expirations > and controller failover. Broker 49554 was the controller and hosted > some partition(s) of LAX1-GRIFFIN-

Re: Issue with topic deletion

2015-02-04 Thread Harsha
whats your zookeeper.session.timeout.ms value On Wed, Feb 4, 2015, at 09:35 PM, Sumit Rangwala wrote: > On Wed, Feb 4, 2015 at 6:14 PM, Joel Koshy wrote: > > > I took a look at your logs. I agree with Harsh that the logs seem > > truncated. The basic issue though is that you have session

Re: Issue with topic deletion

2015-02-04 Thread Sumit Rangwala
On Wed, Feb 4, 2015 at 10:16 PM, Harsha wrote: > >whats your zookeeper.session.timeout.ms value > 3 (30sec) Sumit > > On Wed, Feb 4, 2015, at 09:35 PM, Sumit Rangwala wrote: > > On Wed, Feb 4, 2015 at 6:14 PM, Joel Koshy wrote: > > > > > I took a look at your logs. I agree with

Re: Kafka producer perf script throw java.io.IOException

2015-02-04 Thread Xinyi Su
Hi, I need to get more metrics from csv reporter. If turn off csv-reporter, few output is shown. Thanks. Xinyi On 5 February 2015 at 13:09, tao xiao wrote: > Hi, > > In order to get it work you can turn off csv-reporter. > > On Thu, Feb 5, 2015 at 1:06 PM, Xinyi Su wrote: > > > Hi, > > > > T

Re: Kafka producer perf script throw java.io.IOException

2015-02-04 Thread Jaikiran Pai
>> java.io.IOException: Unable to create /tmp/PerfTopic22_1/ProducerRequestSize.csv It looks like a file with that exact same name already exists which is causing that file creation request to fail. This indicates that probably the metric name (ProducerRequestSize) from which the file is creat

Re: Issue with topic deletion

2015-02-04 Thread Sumit Rangwala
> > Any idea why you have session expirations? This is typically due to GC >> and/or flaky network. Regardless, we should be handling that scenario >> as well. However, your logs seem incomplete. Can you redo this and >> perhaps keep the set up running a little longer and send over those >> logs? >

Re: Kafka producer perf script throw java.io.IOException

2015-02-04 Thread Xinyi Su
Hi, You are right. The file is created by Kafka and Kafka producer csv reporter append metrics continuously during perf test. It should not be created continuously during test but create once only. Another thing is when I turn on "--new-producer" and "--sync" option, nothing is appended into Pro