No partition metadata for topic due to kafka.common.LeaderNotAvailableException

2014-06-04 Thread rafeeq s
Hi , I am using kafka 0.8.1 version and I am getting *LeaderNotAvailableException * frequently. All brokers are up and running well. Note: Recently I have delete a topic. after deleting topic i tried with new topic but also it giving error as LeaderNotAvailableException. Please help to resolve

deleting topics

2014-06-04 Thread Piotr Jagielski
Hi all, Can anyone explain the current state of deleting topics? We've found delete.topic.enable config property and after setting it to true on 0.8.1.1 deletion seems to work ok. There is also a fix of https://issues.apache.org/jira/browse/KAFKA-1397 in the trunk is it possible that 0.8.2 could

How to upgrade Kafka 0.8.1 to 0.8.1.1

2014-06-04 Thread rafeeq s
I have planned to upgrade Kafka version from 0.8.1 to 0.8.1.1 due major bugs. Is there any procedure link to upgrade Kafka 0.8.1 to 0.8.1.1 ? Please share details to upgrade kafka 0.8.1.1. Thanks in Advance,

Re: Hadoop Summit Meetups

2014-06-04 Thread Jonathan Hodges
Hi Neha, Thanks so much to you and the Kafka team for putting together the meetup. It was very nice and gave people from out of town like us the ability to join in person. We are the guys from Pearson Education and we talked a little about supplying some details on some of our use cases with

Re: How to upgrade Kafka 0.8.1 to 0.8.1.1

2014-06-04 Thread François Langelier
No, it's the same procedure for 0.8.0 to 0.8.1 and 0.8.1 to 0.8.1.1 Bring down broker one by one and upgrade the code then restart it goog_1926396836 https://kafka.apache.org/documentation.html#upgrade François Langelier Étudiant en génie Logiciel - École de Technologie Supérieure

Re: How to upgrade Kafka 0.8.1 to 0.8.1.1

2014-06-04 Thread rafeeq s
Thanks François, Got it. On Wed, Jun 4, 2014 at 5:33 PM, François Langelier f.langel...@gmail.com wrote: No, it's the same procedure for 0.8.0 to 0.8.1 and 0.8.1 to 0.8.1.1 Bring down broker one by one and upgrade the code then restart it goog_1926396836

Re: Data loss detection

2014-06-04 Thread Maung Than
We could not find producer msg rate from the matrices in the JConsole — give us some pointers. Also confirming that the reduction in data is due to Avro encoding and we are calculating what we send to producer rather than the output of serializer encoder. Thanks, Maung On Jun 3, 2014, at

Re: Hadoop Summit Meetups

2014-06-04 Thread Neha Narkhede
Glad it was useful. It will be great if you can share your requirements on atomicity. A couple of us are very interested in thinking about transactional messaging in Kafka. Thanks, Neha On Jun 4, 2014 6:57 AM, Jonathan Hodges hodg...@gmail.com wrote: Hi Neha, Thanks so much to you and the

Re: No partition metadata for topic due to kafka.common.LeaderNotAvailableException

2014-06-04 Thread Guozhang Wang
The delete topic utility does not quite work in 0.8.1, we have a bug fix version 0.8.1.1 for this, could you give it a try? Guozhang On Wed, Jun 4, 2014 at 1:15 AM, rafeeq s rafeeq.ec...@gmail.com wrote: Hi , I am using kafka 0.8.1 version and I am getting *LeaderNotAvailableException *

Re: No partition metadata for topic due to kafka.common.LeaderNotAvailableException

2014-06-04 Thread Guozhang Wang
Sorry I was a bit misleading, in 0.8.1.1 we actually turned off the delete topic nob by default. We are working on 0.8.2 which should have a working delete topic functionality. On Wed, Jun 4, 2014 at 9:50 AM, Guozhang Wang wangg...@gmail.com wrote: The delete topic utility does not quite work

Re: deleting topics

2014-06-04 Thread Guozhang Wang
The delete topic utility should be workable in the next release 0.8.2. Guozhang On Wed, Jun 4, 2014 at 1:29 AM, Piotr Jagielski pjagiel...@o2.pl wrote: Hi all, Can anyone explain the current state of deleting topics? We've found delete.topic.enable config property and after setting it to

are consumer offsets stored in a log?

2014-06-04 Thread S Ahmed
I swear I read that Jay Kreps wrote somewhere that consumers now write their offsets in a logfile (not in zookeeper). Is this true or did I misread? Sorry I can't find the article I was reading.

Re: How to parse some of JMX Bean's names.

2014-06-04 Thread Guozhang Wang
Hello Vladimir, comments in-lined. On Wed, Jun 4, 2014 at 6:08 AM, Vladimir Tretyakov vladimir.tretya...@sematext.com wrote: Hi again, few more questions from me: *1.* What I see in JMX:

Re: are consumer offsets stored in a log?

2014-06-04 Thread Neha Narkhede
Consumers in a future release (0.8.2) or trunk will write offsets to a Kafka topic, not zookeeper. Thanks, Neha On Jun 4, 2014 10:00 AM, S Ahmed sahmed1...@gmail.com wrote: I swear I read that Jay Kreps wrote somewhere that consumers now write their offsets in a logfile (not in zookeeper).

Re: [DISCUSS] Kafka Security Specific Features

2014-06-04 Thread Jay Kreps
Hey Joe, Thanks for kicking this discussion off! I totally agree that for something that acts as a central message broker security is critical feature. I think a number of people have been interested in this topic and several people have put effort into special purpose security efforts. Since

Re: are consumer offsets stored in a log?

2014-06-04 Thread Guozhang Wang
Yes, we are migrating the offset management from ZK to the broker as a special log. https://cwiki.apache.org/confluence/display/KAFKA/Inbuilt+Consumer+Offset+Management The code is in trunk, and it is running in production at LinkedIn now. Guozhang On Wed, Jun 4, 2014 at 10:00 AM, S Ahmed

Unclean kafka shutdown

2014-06-04 Thread Ryan Williams
I've noticed that kafka frequently fails to shut down completely when running the shutdown script. After executing kafka-server-stop.sh, it outputs this and does not return to the prompt. Is this anything to be concerned about? An addition kill command will force it down. [2014-06-04

Re: Unclean kafka shutdown

2014-06-04 Thread Mingtao Zhang
Just curious about, whether the memory allocated is released or not. Mingtao Sent from iPhone On Jun 4, 2014, at 1:31 PM, Ryan Williams rwilli...@gmail.com wrote: I've noticed that kafka frequently fails to shut down completely when running the shutdown script. After executing

Re: Unclean kafka shutdown

2014-06-04 Thread Ryan Williams
I will check next time I see it. On Wed, Jun 4, 2014 at 10:37 AM, Mingtao Zhang mail2ming...@gmail.com wrote: Just curious about, whether the memory allocated is released or not. Mingtao Sent from iPhone On Jun 4, 2014, at 1:31 PM, Ryan Williams rwilli...@gmail.com wrote: I've noticed

Fwd: Hadoop Summit Meetups

2014-06-04 Thread Jonathan Hodges
Sorry didn't realize the mailing list wasn't copied... -- Forwarded message -- From: Jonathan Hodges hodg...@gmail.com Date: Wed, Jun 4, 2014 at 10:56 AM Subject: Re: Hadoop Summit Meetups To: Neha Narkhede neha.narkh...@gmail.com We have a number of customer facing online

Re: Unclean kafka shutdown

2014-06-04 Thread Ben Speakmon
Dunno if this is relevant, but if you're spinning down a broker handling live data, you probably want to check this page out: https://cwiki.apache.org/confluence/display/KAFKA/Replication+tools#Replicationtools-1.ControlledShutdown On Wed, Jun 4, 2014 at 10:39 AM, Ryan Williams

Re: How to parse some of JMX Bean's names.

2014-06-04 Thread Guozhang Wang
I think we will make this change in the new consumer, which may be released in 0.9. Guozhang On Wed, Jun 4, 2014 at 12:13 PM, Vladimir Tretyakov vladimir.tretya...@sematext.com wrote: Hi, thx Guozhang Wang, looking forward. When do you think this changes will available? 0.8.2? Jul 2014

Re: Data loss detection

2014-06-04 Thread Jun Rao
It should be sth like clientId-MessagesPerSec. Thanks, Jun On Wed, Jun 4, 2014 at 9:35 AM, Maung Than maung_t...@apple.com wrote: We could not find producer msg rate from the matrices in the JConsole — give us some pointers. Also confirming that the reduction in data is due to Avro

Re: How to parse some of JMX Bean's names.

2014-06-04 Thread Otis Gospodnetic
Hi Guozhang, Since the new consumer is 2-3 months out ... hm, no, looks like 4 months out - October - https://cwiki.apache.org/confluence/display/KAFKA/Future+release+plan Any way you could change this in 0.8.1.2 or 0.8.2? We can submit a patch. Thanks, Otis -- Performance Monitoring * Log

Re-installation of Kafka cluster

2014-06-04 Thread Shaikh Ahmed
Hi, We have Kafka version 0.8.1 cluster installed on our 11 servers. we would like to discard the entire cluster and do the re-installation. Please help me with the steps to achieve this exercise. Thanks in advance. Regards, Riyaz

Unable to build kafka.

2014-06-04 Thread Abhishek Bhattacharjee
I followed the readme for kafka-0.8.0. After I am getting the following warnings ./sbt assembly-package-dependency [info] Loading project definition from /home/abhishek/Project/kafka-0.8.0-src/project [warn] Multiple resolvers having different access mechanism configured with same name

Re: Unable to build kafka.

2014-06-04 Thread Joe Stein
The build looks fine your error seems valid maybe like your broker(s) is(are) not up. /*** Joe Stein Founder, Principal Consultant Big Data Open Source Security LLC http://www.stealth.ly Twitter: @allthingshadoop

Re: [DISCUSS] Kafka Security Specific Features

2014-06-04 Thread Joe Stein
I like the idea of working on the spec and prioritizing. I will update the wiki. - Joestein On Wed, Jun 4, 2014 at 1:11 PM, Jay Kreps jay.kr...@gmail.com wrote: Hey Joe, Thanks for kicking this discussion off! I totally agree that for something that acts as a central message broker

Re: Re-installation of Kafka cluster

2014-06-04 Thread Neha Narkhede
If you don't care about preserving the old data, the steps to re-install would be very similar to the initial install. You will have to delete the existing zookeeper namespace and then set up the new cluster to point to it. Alternatively, you can just use a new zookeeper namespace and treat it

Re: How to parse some of JMX Bean's names.

2014-06-04 Thread Neha Narkhede
Agree that 0.9 is a few months out. 0.8.2, on the other hand, is probably less than a month away. On one hand, we are trying our best to minimize the set of changes to the existing consumer in the interest of saving time to work on the new consumer. Since the old consumer is very complex and is

Re: are consumer offsets stored in a log?

2014-06-04 Thread Neha Narkhede
At LinkedIn, we are just beginning to move our applications to Kafka based offset management. I'm not sure if Todd/Clark/Joel have any numbers to share right away, but depending on the number of consumers you have and the number of topics those consumers pull, it will likely make a sizable impact