Re: kafka 0.8.1.1 log.retention.minutes NOT being honored

2014-07-08 Thread Virendra Pratap Singh
That's correct. The server where in I was running 0.8.1.1 was not honoring this parameter, despite the fact it was set in it server.properties. Not sure if this fact would play any role, the server which was running 0.8.0 was the leader for all the topics and partition in my setup. And the second s

Re: status of 0.8.2

2014-07-08 Thread Jun Rao
Yes, 0.8.2 is compatible with 0.8.0 and 0.8.1 in terms of wire protocols and the upgrade can be done in place. Thanks, Jun On Tue, Jul 8, 2014 at 11:49 AM, Michal Michalski < michal.michal...@boxever.com> wrote: > One more question regarding 0.8.2 - is it planned to be a in-place, > no-downtim

Re: Too Many Open Files Broker Error

2014-07-08 Thread Jun Rao
Does your test program run as the same user as Kafka broker? Thanks, Jun On Tue, Jul 8, 2014 at 1:42 PM, Lung, Paul wrote: > Hi Guys, > > I’m seeing the following errors from the 0.8.1.1 broker. This occurs most > often on the Controller machine. Then the controller process crashes, and > the

Re: Facing issues with Kafka 0.8.1.1 and kafka-reassign-partitions.sh

2014-07-08 Thread Jun Rao
Since you have replication factor 3 and only 3 brokers, you can't move data around in the existing cluster since a partition can have at most replica on each broker. You will need to add the new brokers in first before running the reassignment tool. Thanks, Jun On Tue, Jul 8, 2014 at 10:26 AM,

Re: can the replication factor for a topic be changed after it's created?

2014-07-08 Thread gmail
hi, it seems we can do it now. in official documents: http://kafka.apache.org/documentation.html#basic_ops_increase_replication_factor

RE: How recover leader when broker restart

2014-07-08 Thread chenlax
thank you Guozhang,;i don't know why the isr==null and i can't reappear it, all brokers are working,and add new topic succesfully. Thanks, Lax > Date: Tue, 8 Jul 2014 11:29:05 -0700 > Subject: Re: How recover leader when broker restart > From: wangg...@gmail.com > To: users@kafka.apache.org > >

Re: status of 0.8.2

2014-07-08 Thread Joe Stein
Yup, that was my bad. I updated the comment to explain how the error is happening. /*** Joe Stein Founder, Principal Consultant Big Data Open Source Security LLC http://www.stealth.ly Twitter: @allthingshadoop *

New Consumer Design

2014-07-08 Thread Guozhang Wang
Hi All, We have written a wiki a few weeks back proposing a single-threaded ZK-free consumer client design for 0.9: https://cwiki.apache.org/confluence/display/KAFKA/Kafka+0.9+Consumer+Rewrite+Design We want to share some of the ideas that came up for this design to get some early feedback. The

Re: kafka 0.8.1.1 log.retention.minutes NOT being honored

2014-07-08 Thread Guozhang Wang
Yeah we probably should do that. I am not a committer though so someone else may help fix this issue? On Tue, Jul 8, 2014 at 2:10 PM, Jason Rosenberg wrote: > Ah, ok.it's just no longer documented as such in the config docs? > > > On Tue, Jul 8, 2014 at 4:46 PM, Guozhang Wang wrote: > > >

Re: producer performance

2014-07-08 Thread Guozhang Wang
The second script is using the new producer which allows sending multiple in-flight requests whereas the first script use the current (old) producer which sends only one request at a time to a certain broker. The new producer will be officially released in 0.8.2. Guozhang On Tue, Jul 8, 2014 at

producer performance

2014-07-08 Thread Chen Song
While testing kafka producer performance, I found 2 testing scripts. 1) performance testing script in kafka distribution bin/kafka-producer-perf-test.sh --broker-list localhost:9092 --messages 1000 --topic test --threads 10 --message-size 100 --batch-size 1 --compression-codec 1 2) perfo

Re: kafka 0.8.1.1 log.retention.minutes NOT being honored

2014-07-08 Thread Jason Rosenberg
Ah, ok.it's just no longer documented as such in the config docs? On Tue, Jul 8, 2014 at 4:46 PM, Guozhang Wang wrote: > Jason, > > getLogRetentionTimeMillis() take either "log.retention.minutes" or > "log.retention.hours" and transform the value into milis. So you can > specify using eithe

Re: kafka 0.8.1.1 log.retention.minutes NOT being honored

2014-07-08 Thread Guozhang Wang
Jason, getLogRetentionTimeMillis() take either "log.retention.minutes" or "log.retention.hours" and transform the value into milis. So you can specify using either granularity. Guozhang On Tue, Jul 8, 2014 at 1:11 PM, Jason Rosenberg wrote: > On a related note, in doing the upgrade from 0.8.0

Re: Too Many Open Files Broker Error

2014-07-08 Thread Lung, Paul
Hit the send button too fast. I verified the number of open file descriptors from the broker by using ³sudo lsof -p², and by using ³sudo ls -l /proc//fd | wc -l². Paul On 7/8/14, 1:42 PM, "Lung, Paul" wrote: >Hi Guys, > >I¹m seeing the following errors from the 0.8.1.1 broker. This occurs most

Too Many Open Files Broker Error

2014-07-08 Thread Lung, Paul
Hi Guys, I’m seeing the following errors from the 0.8.1.1 broker. This occurs most often on the Controller machine. Then the controller process crashes, and the controller bounces to other machines, which causes those machines to crash. Looking at the file descriptors being held by the process,

Re: Zookeeper Version Mismatch Problem In 0.8.1

2014-07-08 Thread Lung, Paul
Thank you. Paul On 7/8/14, 11:41 AM, "Guozhang Wang" wrote: >I think you are hitting > >https://issues.apache.org/jira/browse/KAFKA-1382 > >The fix is committed to trunk already, so in the next release you should >not see this issue any more. > >Guozhang > > >On Tue, Jul 8, 2014 at 10:29 AM, Lu

Re: kafka 0.8.1.1 log.retention.minutes NOT being honored

2014-07-08 Thread Jason Rosenberg
On a related note, in doing the upgrade from 0.8.0, I noticed that the config property changed from 'log.retention.hours' to 'log.retention.minutes'. Would it have made more sense to deprecate rather than replace there? Also, I notice that internally, in the KafkaConfig class, it's represented as

Re: kafka 0.8.1.1 log.retention.minutes NOT being honored

2014-07-08 Thread Guozhang Wang
Server properties should affect on only the local instance separately. Are you saying the property is not honored even on the 0.8.1 machines? Guozhang On Mon, Jul 7, 2014 at 3:55 PM, Virendra Pratap Singh < vpsi...@yahoo-inc.com.invalid> wrote: > By setting this property > log.retention.mins=10

Re: Monitoring Producers at Large Scale

2014-07-08 Thread Bhavesh Mistry
HI Otis, You are right. If the Kafka itself have problem (QUEUE is full, auto rebalance etc, drop event), how can it transmit the logs... So we have tried to avoid "agent based" solution Apache Flume Agent or Syslog configuration. You are right we have to build a redundant transportation for mo

Re: status of 0.8.2

2014-07-08 Thread Michal Michalski
One more question regarding 0.8.2 - is it planned to be a in-place, no-downtime release (I'm using 0.8.0 now)? By looking at the version number changes only I'd guess it is, but... ;-) Michal Kind regards, Michał Michalski, michal.michal...@boxever.com On 8 July 2014 18:22, Joel Koshy wrote:

Re: Zookeeper Version Mismatch Problem In 0.8.1

2014-07-08 Thread Guozhang Wang
I think you are hitting https://issues.apache.org/jira/browse/KAFKA-1382 The fix is committed to trunk already, so in the next release you should not see this issue any more. Guozhang On Tue, Jul 8, 2014 at 10:29 AM, Lung, Paul wrote: > Hi All, > > > I’m seeing the following issues after run

Re: How recover leader when broker restart

2014-07-08 Thread Guozhang Wang
Lax, Under that scenario you would better first fix the issue of isr==null by checking if anything went wrong on the brokers. Guozhang On Mon, Jul 7, 2014 at 8:43 PM, chenlax wrote: > use preferred tool can rebalance leadership,but if the isr are null then > the leader is only -1,how i can re

Re: Facing issues with Kafka 0.8.1.1 and kafka-reassign-partitions.sh

2014-07-08 Thread Florian Dambrine
I realized that I did not respond to you Clark, Here is the entire Json that I sliced in multiple pieces: {"version":1,"partitions":[{"topic":"SLOTS","partition":20,"replicas":[101421743,105114483,101461702]},{"topic":"RTB","partition":12,"replicas":[101671664,101812541,101862816]},{"topic":"B_IM

Re: Facing issues with Kafka 0.8.1.1 and kafka-reassign-partitions.sh

2014-07-08 Thread Florian Dambrine
Here is the entire logic to rebalance the cluster which is done by this groovy script ( https://github.com/Lowess/Kafka/blob/master/KafkaPartitionRebalancer.groovy) #1: Request the zookeeper and get the broker id list #2: Request zookeeper and get the list of topic #3: Generate the topic-to-move.j

Re: Facing issues with Kafka 0.8.1.1 and kafka-reassign-partitions.sh

2014-07-08 Thread Clark Haskins
Can you copy/paste the json you are passing to the reassignment tool? Plus the commands. Also do a describe on your topics. -Clark Clark Elliott Haskins III LinkedIn DDS Site Reliability Engineer Kafka, Zookeeper, Samza SRE Mobile: 505.385.1484 BlueJeans: https://www.bluejeans.com/chaskins chas

Zookeeper Version Mismatch Problem In 0.8.1

2014-07-08 Thread Lung, Paul
Hi All, I’m seeing the following issues after running a Kafka broker cluster for 2 months: [2014-07-07 17:54:19,798] ERROR Conditional update of path /brokers/topics/mini__065active_80__32__miniactiveitembrn_chd_qn/partitions/0/state with data {"controller_epoch":9,"leader":2129087,

Re: Facing issues with Kafka 0.8.1.1 and kafka-reassign-partitions.sh

2014-07-08 Thread Florian Dambrine
I let the tool running for an entire weekend on the test cluster and on Monday it was still saying "failed"... I have 500 Go per Kafka node and it is a 8 nodes cluster. I am also wondering if I am using the tool correctly. Currently I am running the tool to rebalance everything across the entire

Re: status of 0.8.2

2014-07-08 Thread Joel Koshy
Hi Joe, I had a question for you in that RB since I don't fully understand it. Maybe you can help clarify how the fix works. Thanks, Joel On Tue, Jul 08, 2014 at 10:55:56AM -0400, Joe Stein wrote: > I wrote it, so I can't commit it without other committers agreeing. > > Last I recall I updated

Re: status of 0.8.2

2014-07-08 Thread Joe Stein
I think it should be ok that code as I recall is isolated. /*** Joe Stein Founder, Principal Consultant Big Data Open Source Security LLC http://www.stealth.ly Twitter: @allthingshadoop **

Re: status of 0.8.2

2014-07-08 Thread Jason Rosenberg
Thanks Joe, I'll let you know what I find. Do you anticipate any issues with it working in 0.8.1.1? Jason On Tue, Jul 8, 2014 at 10:55 AM, Joe Stein wrote: > I wrote it, so I can't commit it without other committers agreeing. > > Last I recall I updated the patch from the feedback in the revi

Re: status of 0.8.2

2014-07-08 Thread Joe Stein
I wrote it, so I can't commit it without other committers agreeing. Last I recall I updated the patch from the feedback in the reviewboard but haven't looked at it in months. I am glad though it resolved the issue you were having and we can figure how to get the patch to work with 0.8.1.1 if you

Re: status of 0.8.2

2014-07-08 Thread Jason Rosenberg
Is there a blocker to getting the patch for kafka-1180 applied? Is the patch for 0.8.0 no longer compatible for trunk? I'm actually going to see if I can get it to work for 0.8.1.1 today. Thanks, Jason On Mon, Jul 7, 2014 at 9:41 PM, Jun Rao wrote: > Two biggest features in 0.8.2 are Kafka-